diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index 8829072680..99c7da15a3 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -564,7 +564,7 @@ public: // Force is required if there is not a restorable snapshot which both // - begins at or after expireEndVersion // - ends at or before restorableBeginVersion - bool forceNeeded = true; + state bool forceNeeded = true; for(KeyspaceSnapshotFile &s : desc.snapshots) { if(s.restorable.orDefault(false) && s.beginVersion >= expireEndVersion && s.endVersion <= restorableBeginVersion) { forceNeeded = false; @@ -572,9 +572,6 @@ public: } } - if(forceNeeded && !force) - throw backup_cannot_expire(); - // Get metadata state Optional expiredEnd; state Optional logBegin; @@ -604,52 +601,94 @@ public: newLogBeginVersion = expireEndVersion; } else { - // If the last log overlaps the expiredEnd then use the log's begin version + // If the last log overlaps the expiredEnd then use the log's begin version and move the expiredEnd + // back to match it. if(last.endVersion > expireEndVersion) { newLogBeginVersion = last.beginVersion; logs.pop_back(); + expireEndVersion = newLogBeginVersion.get(); } } } - // If we have a new log begin version then potentially update the property but definitely set - // expireEndVersion to the new log begin because we will only be deleting files up to but not - // including that version. - if(newLogBeginVersion.present()) { - expireEndVersion = newLogBeginVersion.get(); - // If the new version is greater than the existing one or the - // existing one is not present then write the new value - if(logBegin.orDefault(0) < newLogBeginVersion.get()) { - Void _ = wait(bc->logBeginVersion().set(newLogBeginVersion.get())); - } - } - else { - // Otherwise, if the old logBeginVersion is present and older than expireEndVersion then clear it because - // it refers to a version in a range we're about to delete and apparently continuity through - // expireEndVersion is broken. - if(logBegin.present() && logBegin.get() < expireEndVersion) - Void _ = wait(bc->logBeginVersion().clear()); - } - - // Delete files - state std::vector> deletes; + // Make a list of files to delete + state std::vector toDelete; + // Move filenames out of vector then destroy it to save memory for(auto const &f : logs) { - deletes.push_back(bc->deleteFile(f.fileName)); + toDelete.push_back(std::move(f.fileName)); } + logs.clear(); + // Move filenames out of vector then destroy it to save memory for(auto const &f : ranges) { // Must recheck version because list returns data up to and including the given endVersion if(f.version < expireEndVersion) - deletes.push_back(bc->deleteFile(f.fileName)); + toDelete.push_back(std::move(f.fileName)); } + ranges.clear(); for(auto const &f : desc.snapshots) { if(f.endVersion < expireEndVersion) - deletes.push_back(bc->deleteFile(f.fileName)); + toDelete.push_back(std::move(f.fileName)); + } + desc = BackupDescription(); + + // If some files to delete were found AND force is needed AND the force option is NOT set, then fail + if(!toDelete.empty() && forceNeeded && !force) + throw backup_cannot_expire(); + + // We are about to start deleting files, at which point no data prior to the expire end version can be + // safely assumed to exist. The [logBegin, logEnd) range from the container's metadata describes + // a range of log versions which can be assumed to exist, so if the range of data being deleted overlaps + // that range then the metadata range must be updated. + + // If we're expiring the entire log range described by the metadata then clear both metadata values + if(logEnd.present() && logEnd.get() < expireEndVersion) { + if(logBegin.present()) + Void _ = wait(bc->logBeginVersion().clear()); + if(logEnd.present()) + Void _ = wait(bc->logEndVersion().clear()); + } + else { + // If we are expiring to a point within the metadata range then update the begin if we have a new + // log begin version (which we should!) or clear the metadata range if we do not (which would be + // repairing the metadata from an incorrect state) + if(logBegin.present() && logBegin.get() < expireEndVersion) { + if(newLogBeginVersion.present()) { + Void _ = wait(bc->logBeginVersion().set(newLogBeginVersion.get())); + } + else { + if(logBegin.present()) + Void _ = wait(bc->logBeginVersion().clear()); + if(logEnd.present()) + Void _ = wait(bc->logEndVersion().clear()); + } + } } - Void _ = wait(waitForAll(deletes)); + // Delete files, but limit parallelism because the file list could use a lot of memory and the corresponding + // delete actor states would use even more if they all existed at the same time. + state std::list> deleteFutures; + + while(!toDelete.empty() || !deleteFutures.empty()) { + + // While there are files to delete and budget in the deleteFutures list, start a delete + while(!toDelete.empty() && deleteFutures.size() < CLIENT_KNOBS->BACKUP_CONCURRENT_DELETES) { + deleteFutures.push_back(bc->deleteFile(toDelete.back())); + toDelete.pop_back(); + } + + // Wait for deletes to finish until there are only targetDeletesInFlight remaining. + // If there are no files left to start then this value is 0, otherwise it is one less + // than the delete concurrency limit. + state int targetFuturesSize = toDelete.empty() ? 0 : (CLIENT_KNOBS->BACKUP_CONCURRENT_DELETES - 1); + + while(deleteFutures.size() > targetFuturesSize) { + Void _ = wait(deleteFutures.front()); + deleteFutures.pop_front(); + } + } // Update the expiredEndVersion property. Void _ = wait(bc->expiredEndVersion().set(expireEndVersion)); diff --git a/fdbclient/Knobs.cpp b/fdbclient/Knobs.cpp index 6c6232089f..e22f4550e2 100644 --- a/fdbclient/Knobs.cpp +++ b/fdbclient/Knobs.cpp @@ -91,6 +91,7 @@ ClientKnobs::ClientKnobs(bool randomize) { init( TASKBUCKET_MAX_TASK_KEYS, 1000 ); if( randomize && BUGGIFY ) TASKBUCKET_MAX_TASK_KEYS = 20; //Backup + init( BACKUP_CONCURRENT_DELETES, 100 ); init( BACKUP_SIMULATED_LIMIT_BYTES, 1e6 ); if( randomize && BUGGIFY ) BACKUP_SIMULATED_LIMIT_BYTES = 1000; init( BACKUP_GET_RANGE_LIMIT_BYTES, 1e6 ); init( BACKUP_LOCK_BYTES, 1e8 ); diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index d24c6717c6..1ce0cec1ea 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -93,6 +93,7 @@ public: int TASKBUCKET_MAX_TASK_KEYS; // Backup + int BACKUP_CONCURRENT_DELETES; int BACKUP_SIMULATED_LIMIT_BYTES; int BACKUP_GET_RANGE_LIMIT_BYTES; int BACKUP_LOCK_BYTES; diff --git a/fdbrpc/AsyncFileEIO.actor.h b/fdbrpc/AsyncFileEIO.actor.h index a857df8aa3..3205cf9800 100644 --- a/fdbrpc/AsyncFileEIO.actor.h +++ b/fdbrpc/AsyncFileEIO.actor.h @@ -83,7 +83,7 @@ public: TraceEvent(notFound ? SevWarn : SevWarnAlways, "FileOpenError").error(e).GetLastError().detail("File", filename).detail("Flags", flags).detail("Mode", mode); throw e; } - TraceEvent("AsyncFileOpened").detail("Filename", filename).detail("fd", r->result).detail("Flags", flags); + TraceEvent("AsyncFileOpened").detail("Filename", filename).detail("fd", r->result).detail("Flags", flags).suppressFor(1.0); if ((flags & OPEN_LOCK) && !lock_fd(r->result)) { TraceEvent(SevError, "UnableToLockFile").detail("filename", filename).GetLastError(); @@ -264,7 +264,7 @@ private: state eio_req* r = eio_close(fd, 0, eio_callback, &p); Void _ = wait( p.getFuture() ); if (r->result) error( "CloseError", fd, r ); - TraceEvent("AsyncFileClosed").detail("fd", fd); + TraceEvent("AsyncFileClosed").detail("fd", fd).suppressFor(1.0); } ACTOR static Future read_impl( int fd, void* data, int length, int64_t offset ) { diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index fc8ad74b55..79894aef0c 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -152,7 +152,7 @@ public: }; bool workerAvailable( WorkerInfo const& worker, bool checkStable ) { - return IFailureMonitor::failureMonitor().getState(worker.interf.storage.getEndpoint()).isAvailable() && ( !checkStable || worker.reboots < 2 ); + return ( now() - startTime < 2 * FLOW_KNOBS->SERVER_REQUEST_INTERVAL ) || ( IFailureMonitor::failureMonitor().getState(worker.interf.storage.getEndpoint()).isAvailable() && ( !checkStable || worker.reboots < 2 ) ); } std::pair getStorageWorker( RecruitStorageRequest const& req ) { diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 03124d9a1b..eedaac2fc5 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -611,7 +611,6 @@ struct DDTeamCollection { } if( foundExact || (req.wantsTrueBest && bestOption.present() ) ) { - TraceEvent("getTeam").detail("wantsVariety", req.wantsNewServers).detail("bestOption", bestOption.get()->getDesc()); ASSERT( bestOption.present() ); req.reply.send( bestOption ); return Void(); diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index 8a13b52fd7..db6210a8c1 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -702,7 +702,7 @@ ACTOR Future dataDistributionRelocator( DDQueueData *self, RelocateData rd destination->addDataInFlightToTeam( +metrics.bytes ); - TraceEvent("RelocateShardHasDestination", masterId) + TraceEvent(relocateShardInterval.severity, "RelocateShardHasDestination", masterId) .detail("PairId", relocateShardInterval.pairID) .detail("DestinationTeam", destination->getDesc()); diff --git a/fdbserver/LeaderElection.actor.cpp b/fdbserver/LeaderElection.actor.cpp index 0bccf08dc2..8cb90f7d00 100644 --- a/fdbserver/LeaderElection.actor.cpp +++ b/fdbserver/LeaderElection.actor.cpp @@ -82,6 +82,10 @@ ACTOR Future tryBecomeLeaderInternal( ServerCoordinators coordinators, Val state bool iAmLeader = false; state UID prevChangeID; + if( asyncProcessClass->get().machineClassFitness(ProcessClass::ClusterController) > ProcessClass::UnsetFit || asyncIsExcluded->get() ) { + Void _ = wait( delay(SERVER_KNOBS->WAIT_FOR_GOOD_RECRUITMENT_DELAY) ); + } + nominees->set( vector>( coordinators.clientLeaderServers.size() ) ); myInfo.serializedInfo = proposedSerializedInterface; diff --git a/fdbserver/Status.actor.cpp b/fdbserver/Status.actor.cpp index fd1bf97df0..f218de272e 100644 --- a/fdbserver/Status.actor.cpp +++ b/fdbserver/Status.actor.cpp @@ -790,8 +790,21 @@ ACTOR static Future processStatusFetcher( if (programStarts.count(address)) { auto const& psxml = programStarts.at(address); - int64_t memLimit = parseInt64(extractAttribute(psxml, "MemoryLimit")); - memoryObj["limit_bytes"] = memLimit; + + if(psxml.size() > 0) { + int64_t memLimit = parseInt64(extractAttribute(psxml, "MemoryLimit")); + memoryObj["limit_bytes"] = memLimit; + + std::string version; + if (tryExtractAttribute(psxml, LiteralStringRef("Version"), version)) { + statusObj["version"] = version; + } + + std::string commandLine; + if (tryExtractAttribute(psxml, LiteralStringRef("CommandLine"), commandLine)) { + statusObj["command_line"] = commandLine; + } + } } // if this process address is in the machine metrics @@ -811,9 +824,10 @@ ACTOR static Future processStatusFetcher( StatusArray messages; - if (errors.count(address) && errors[address].size()) + if (errors.count(address) && errors[address].size()) { // returns status object with type and time of error messages.push_back(getError(errors.at(address))); + } // string of address used so that other fields of a NetworkAddress are not compared std::string strAddress = address.toString(); @@ -838,18 +852,6 @@ ACTOR static Future processStatusFetcher( // Get roles for the worker's address as an array of objects statusObj["roles"] = roles.getStatusForAddress(address); - if (programStarts.count(address)) { - auto const& psxml = programStarts.at(address); - - std::string version; - if (tryExtractAttribute(psxml, LiteralStringRef("Version"), version)) - statusObj["version"] = version; - - std::string commandLine; - if (tryExtractAttribute(psxml, LiteralStringRef("CommandLine"), commandLine)) - statusObj["command_line"] = commandLine; - } - if (configuration.present()){ statusObj["excluded"] = configuration.get().isExcludedServer(address); } diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index f6a778ef32..99b1af91d8 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -837,7 +837,7 @@ struct ConsistencyCheckWorkload : TestWorkload else if(!isRelocating) { TraceEvent("ConsistencyCheck_StorageServerUnavailable").detail("StorageServer", storageServers[j]).detail("ShardBegin", printable(range.begin)).detail("ShardEnd", printable(range.end)) - .detail("Address", storageServerInterfaces[j].address()).detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token); + .detail("Address", storageServerInterfaces[j].address()).detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token).suppressFor(1.0); //All shards should be available in quiscence if(self->performQuiescentChecks) @@ -977,7 +977,9 @@ struct ConsistencyCheckWorkload : TestWorkload } } - TraceEvent("ConsistencyCheck_ReadRange").detail("range", printable(range)).detail("bytesRead", bytesReadInRange); + if(bytesReadInRange > 0) { + TraceEvent("ConsistencyCheck_ReadRange").detail("range", printable(range)).detail("bytesRead", bytesReadInRange); + } } //SOMEDAY: when background data distribution is implemented, include this test diff --git a/packaging/msi/FDBInstaller.wxs b/packaging/msi/FDBInstaller.wxs index e49398eb30..f1995446d2 100644 --- a/packaging/msi/FDBInstaller.wxs +++ b/packaging/msi/FDBInstaller.wxs @@ -32,7 +32,7 @@ ","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} + schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"queue_disk_available_bytes":12341234,"persistent_disk_total_bytes":12341234,"kvstore_available_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} testName=RandomClogging testDuration=30.0 diff --git a/tests/rare/LargeApiCorrectnessStatus.txt b/tests/rare/LargeApiCorrectnessStatus.txt index fbb67df02d..8e4195a140 100644 --- a/tests/rare/LargeApiCorrectnessStatus.txt +++ b/tests/rare/LargeApiCorrectnessStatus.txt @@ -24,4 +24,4 @@ testTitle=ApiCorrectnessTest testName=Status testDuration=30.0 - schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"persistent_disk_total_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} + schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"queue_disk_available_bytes":12341234,"persistent_disk_total_bytes":12341234,"kvstore_available_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} diff --git a/tests/slow/DDBalanceAndRemoveStatus.txt b/tests/slow/DDBalanceAndRemoveStatus.txt index c256d4f917..ccd6c0bac2 100644 --- a/tests/slow/DDBalanceAndRemoveStatus.txt +++ b/tests/slow/DDBalanceAndRemoveStatus.txt @@ -43,4 +43,4 @@ testTitle=DDBalance_test testName=Status testDuration=30.0 - schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"persistent_disk_total_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} + schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"queue_disk_available_bytes":12341234,"persistent_disk_total_bytes":12341234,"kvstore_available_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}}