Merge pull request #656 from etschannen/feature-trace-event-speedup

Reduce CPU cost of suppressed trace events
This commit is contained in:
Evan Tschannen 2018-08-02 11:07:06 -07:00 committed by GitHub
commit 88ab07a24f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
57 changed files with 329 additions and 348 deletions

View File

@ -369,7 +369,7 @@ int FDBLibTLSSession::handshake() {
case TLS_WANT_POLLOUT:
return WANT_WRITE;
default:
TraceEvent("FDBLibTLSHandshakeError", uid).detail("LibTLSErrorMessage", tls_error(tls_ctx)).suppressFor(1.0, true);
TraceEvent("FDBLibTLSHandshakeError", uid).suppressFor(1.0).detail("LibTLSErrorMessage", tls_error(tls_ctx));
return FAILED;
}
}
@ -389,7 +389,7 @@ int FDBLibTLSSession::read(uint8_t* data, int length) {
return (int)n;
}
if (n == 0) {
TraceEvent("FDBLibTLSReadEOF").suppressFor(1.0, true);
TraceEvent("FDBLibTLSReadEOF").suppressFor(1.0);
return FAILED;
}
if (n == TLS_WANT_POLLIN)
@ -397,7 +397,7 @@ int FDBLibTLSSession::read(uint8_t* data, int length) {
if (n == TLS_WANT_POLLOUT)
return WANT_WRITE;
TraceEvent("FDBLibTLSReadError", uid).detail("LibTLSErrorMessage", tls_error(tls_ctx)).suppressFor(1.0, true);
TraceEvent("FDBLibTLSReadError", uid).suppressFor(1.0).detail("LibTLSErrorMessage", tls_error(tls_ctx));
return FAILED;
}
@ -416,7 +416,7 @@ int FDBLibTLSSession::write(const uint8_t* data, int length) {
return (int)n;
}
if (n == 0) {
TraceEvent("FDBLibTLSWriteEOF", uid).suppressFor(1.0, true);
TraceEvent("FDBLibTLSWriteEOF", uid).suppressFor(1.0);
return FAILED;
}
if (n == TLS_WANT_POLLIN)
@ -424,6 +424,6 @@ int FDBLibTLSSession::write(const uint8_t* data, int length) {
if (n == TLS_WANT_POLLOUT)
return WANT_WRITE;
TraceEvent("FDBLibTLSWriteError", uid).detail("LibTLSErrorMessage", tls_error(tls_ctx)).suppressFor(1.0, true);
TraceEvent("FDBLibTLSWriteError", uid).suppressFor(1.0).detail("LibTLSErrorMessage", tls_error(tls_ctx));
return FAILED;
}

View File

@ -3087,7 +3087,7 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
}
catch(Error &e) {
//options->setOption() prints error message
TraceEvent(SevWarn, "CLISetOptionError").detail("Option", printable(tokens[2])).error(e);
TraceEvent(SevWarn, "CLISetOptionError").error(e).detail("Option", printable(tokens[2]));
is_error = true;
}
}
@ -3138,7 +3138,7 @@ ACTOR Future<int> runCli(CLIOptions opt) {
linenoise.historyLoad(historyFilename);
}
catch(Error &e) {
TraceEvent(SevWarnAlways, "ErrorLoadingCliHistory").detail("Filename", historyFilename.empty() ? "<unknown>" : historyFilename).error(e).GetLastError();
TraceEvent(SevWarnAlways, "ErrorLoadingCliHistory").error(e).detail("Filename", historyFilename.empty() ? "<unknown>" : historyFilename).GetLastError();
}
state int result = wait(cli(opt, &linenoise));
@ -3148,7 +3148,7 @@ ACTOR Future<int> runCli(CLIOptions opt) {
linenoise.historySave(historyFilename);
}
catch(Error &e) {
TraceEvent(SevWarnAlways, "ErrorSavingCliHistory").detail("Filename", historyFilename).error(e).GetLastError();
TraceEvent(SevWarnAlways, "ErrorSavingCliHistory").error(e).detail("Filename", historyFilename).GetLastError();
}
}

View File

@ -793,8 +793,7 @@ public:
Void _ = wait(f->finish());
return Void();
} catch(Error &e) {
if(e.code() != error_code_actor_cancelled)
TraceEvent(SevWarn, "BackupContainerWritePropertyFailed").detail("Path", path).error(e);
TraceEvent(SevWarn, "BackupContainerWritePropertyFailed").error(e).detail("Path", path);
throw;
}
}
@ -816,8 +815,7 @@ public:
} catch(Error &e) {
if(e.code() == error_code_file_not_found)
return Optional<Version>();
if(e.code() != error_code_actor_cancelled)
TraceEvent(SevWarn, "BackupContainerReadPropertyFailed").detail("Path", path).error(e);
TraceEvent(SevWarn, "BackupContainerReadPropertyFailed").error(e).detail("Path", path);
throw;
}
}

View File

@ -596,12 +596,12 @@ namespace fileBackup {
} catch(Error &e) {
TraceEvent(SevWarn, "FileRestoreCorruptRangeFileBlock")
.error(e)
.detail("Filename", file->getFilename())
.detail("BlockOffset", offset)
.detail("BlockLen", len)
.detail("ErrorRelativeOffset", reader.rptr - buf.begin())
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset)
.error(e);
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset);
throw;
}
}
@ -690,12 +690,12 @@ namespace fileBackup {
} catch(Error &e) {
TraceEvent(SevWarn, "FileRestoreCorruptLogFileBlock")
.error(e)
.detail("Filename", file->getFilename())
.detail("BlockOffset", offset)
.detail("BlockLen", len)
.detail("ErrorRelativeOffset", reader.rptr - buf.begin())
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset)
.error(e);
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset);
throw;
}
}
@ -1047,11 +1047,11 @@ namespace fileBackup {
state Key endKey = Params.endKey().get(task);
TraceEvent("FileBackupRangeStart")
.suppressFor(60)
.detail("BackupUID", BackupConfig(task).getUid())
.detail("BeginKey", Params.beginKey().get(task).printable())
.detail("EndKey", Params.endKey().get(task).printable())
.detail("TaskKey", task->key.printable())
.suppressFor(60, true);
.detail("TaskKey", task->key.printable());
// When a key range task saves the last chunk of progress and then the executor dies, when the task continues
// its beginKey and endKey will be equal but there is no work to be done.
@ -1111,14 +1111,14 @@ namespace fileBackup {
bool usedFile = wait(finishRangeFile(outFile, cx, task, taskBucket, KeyRangeRef(beginKey, nextKey), outVersion));
TraceEvent("FileBackupWroteRangeFile")
.suppressFor(60)
.detail("BackupUID", backup.getUid())
.detail("Size", outFile->size())
.detail("Keys", nrKeys)
.detail("ReadVersion", outVersion)
.detail("BeginKey", beginKey.printable())
.detail("EndKey", nextKey.printable())
.detail("AddedFileToMap", usedFile)
.suppressFor(60, true);
.detail("AddedFileToMap", usedFile);
nrKeys = 0;
beginKey = nextKey;
@ -1164,12 +1164,12 @@ namespace fileBackup {
if (nextKey != keys[idx]) {
addTaskVector.push_back(addTask(tr, taskBucket, task, task->getPriority(), nextKey, keys[idx], TaskCompletionKey::joinWith(onDone)));
TraceEvent("FileBackupRangeSplit")
.suppressFor(60)
.detail("BackupUID", BackupConfig(task).getUid())
.detail("BeginKey", Params.beginKey().get(task).printable())
.detail("EndKey", Params.endKey().get(task).printable())
.detail("SliceBeginKey", nextKey.printable())
.detail("SliceEndKey", keys[idx].printable())
.suppressFor(60, true);
.detail("SliceEndKey", keys[idx].printable());
}
nextKey = keys[idx];
}
@ -1197,11 +1197,11 @@ namespace fileBackup {
Void _ = wait(taskBucket->finish(tr, task));
TraceEvent("FileBackupRangeFinish")
.suppressFor(60)
.detail("BackupUID", BackupConfig(task).getUid())
.detail("BeginKey", Params.beginKey().get(task).printable())
.detail("EndKey", Params.endKey().get(task).printable())
.detail("TaskKey", task->key.printable())
.suppressFor(60, true);
.detail("TaskKey", task->key.printable());
return Void();
}
@ -1540,9 +1540,9 @@ namespace fileBackup {
loop {
try {
TraceEvent("FileBackupSnapshotDispatchAddingTasks")
.suppressFor(2)
.detail("TasksToAdd", rangesToAdd.size())
.detail("NewBatchSize", newBatchSize)
.suppressFor(2, true);
.detail("NewBatchSize", newBatchSize);
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
@ -1609,12 +1609,12 @@ namespace fileBackup {
addTaskFutures.push_back(success(BackupRangeTaskFunc::addTask(tr, taskBucket, task, priority, range.begin, range.end, TaskCompletionKey::joinWith(snapshotBatchFuture), Reference<TaskFuture>(), scheduledVersion)));
TraceEvent("FileBackupSnapshotRangeDispatched")
.suppressFor(2)
.detail("BackupUID", config.getUid())
.detail("CurrentVersion", recentReadVersion)
.detail("ScheduledVersion", scheduledVersion)
.detail("BeginKey", range.begin.printable())
.detail("EndKey", range.end.printable())
.suppressFor(2);
.detail("EndKey", range.end.printable());
}
else {
// This shouldn't happen because if the transaction was already done or if another execution
@ -1813,12 +1813,12 @@ namespace fileBackup {
Void _ = wait(outFile->finish());
TraceEvent("FileBackupWroteLogFile")
.suppressFor(60)
.detail("BackupUID", config.getUid())
.detail("Size", outFile->size())
.detail("BeginVersion", beginVersion)
.detail("EndVersion", endVersion)
.detail("LastReadVersion", latestVersion)
.suppressFor(60, true);
.detail("LastReadVersion", latestVersion);
Params.fileSize().set(task, outFile->size());
@ -2017,10 +2017,10 @@ namespace fileBackup {
state Version endVersion = std::max<Version>( tr->getReadVersion().get() + 1, beginVersion + (CLIENT_KNOBS->BACKUP_MAX_LOG_RANGES-1)*CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE );
TraceEvent("FileBackupLogDispatch")
.suppressFor(60)
.detail("BeginVersion", beginVersion)
.detail("EndVersion", endVersion)
.detail("RestorableVersion", restorableVersion.orDefault(-1))
.suppressFor(60, true);
.detail("RestorableVersion", restorableVersion.orDefault(-1));
state Reference<TaskFuture> logDispatchBatchFuture = futureBucket->future(tr);
@ -2040,10 +2040,10 @@ namespace fileBackup {
Void _ = wait(taskBucket->finish(tr, task));
TraceEvent("FileBackupLogsDispatchContinuing")
.suppressFor(60)
.detail("BackupUID", config.getUid())
.detail("BeginVersion", beginVersion)
.detail("EndVersion", endVersion)
.suppressFor(60, true);
.detail("EndVersion", endVersion);
return Void();
}
@ -2437,14 +2437,14 @@ namespace fileBackup {
state int64_t readLen = Params.readLen().get(task);
TraceEvent("FileRestoreRangeStart")
.suppressFor(60)
.detail("RestoreUID", restore.getUid())
.detail("FileName", rangeFile.fileName)
.detail("FileVersion", rangeFile.version)
.detail("FileSize", rangeFile.fileSize)
.detail("ReadOffset", readOffset)
.detail("ReadLen", readLen)
.detail("TaskInstance", (uint64_t)this)
.suppressFor(60, true);
.detail("TaskInstance", (uint64_t)this);
state Reference<ReadYourWritesTransaction> tr( new ReadYourWritesTransaction(cx) );
state Future<Reference<IBackupContainer>> bc;
@ -2554,6 +2554,7 @@ namespace fileBackup {
Void _ = wait(tr->commit());
TraceEvent("FileRestoreCommittedRange")
.suppressFor(60)
.detail("RestoreUID", restore.getUid())
.detail("FileName", rangeFile.fileName)
.detail("FileVersion", rangeFile.version)
@ -2568,8 +2569,7 @@ namespace fileBackup {
.detail("DataSize", data.size())
.detail("Bytes", txBytes)
.detail("OriginalFileRange", printable(originalFileRange))
.detail("TaskInstance", (uint64_t)this)
.suppressFor(60, true);
.detail("TaskInstance", (uint64_t)this);
// Commit succeeded, so advance starting point
start = i;
@ -2650,6 +2650,7 @@ namespace fileBackup {
state int64_t readLen = Params.readLen().get(task);
TraceEvent("FileRestoreLogStart")
.suppressFor(60)
.detail("RestoreUID", restore.getUid())
.detail("FileName", logFile.fileName)
.detail("FileBeginVersion", logFile.version)
@ -2657,8 +2658,7 @@ namespace fileBackup {
.detail("FileSize", logFile.fileSize)
.detail("ReadOffset", readOffset)
.detail("ReadLen", readLen)
.detail("TaskInstance", (uint64_t)this)
.suppressFor(60, true);
.detail("TaskInstance", (uint64_t)this);
state Reference<ReadYourWritesTransaction> tr( new ReadYourWritesTransaction(cx) );
state Reference<IBackupContainer> bc;
@ -2718,6 +2718,7 @@ namespace fileBackup {
Void _ = wait(tr->commit());
TraceEvent("FileRestoreCommittedLog")
.suppressFor(60)
.detail("RestoreUID", restore.getUid())
.detail("FileName", logFile.fileName)
.detail("FileBeginVersion", logFile.version)
@ -2730,8 +2731,7 @@ namespace fileBackup {
.detail("EndIndex", i)
.detail("DataSize", data.size())
.detail("Bytes", txBytes)
.detail("TaskInstance", (uint64_t)this)
.suppressFor(60, true);
.detail("TaskInstance", (uint64_t)this);
// Commit succeeded, so advance starting point
start = i;
@ -2981,10 +2981,10 @@ namespace fileBackup {
beginBlock = 0;
TraceEvent("FileRestoreDispatchedFile")
.suppressFor(60)
.detail("RestoreUID", restore.getUid())
.detail("FileName", f.fileName)
.detail("TaskInstance", (uint64_t)this)
.suppressFor(60, true);
.detail("TaskInstance", (uint64_t)this);
}
// If no blocks were dispatched then the next dispatch task should run now and be joined with the allPartsDone future

View File

@ -99,7 +99,7 @@ bool ClusterConnectionFile::fileContentsUpToDate(ClusterConnectionString &fileCo
return fileConnectionString.toString() == cs.toString();
}
catch (Error& e) {
TraceEvent(SevWarnAlways, "ClusterFileError").detail("Filename", filename).error(e);
TraceEvent(SevWarnAlways, "ClusterFileError").error(e).detail("Filename", filename);
return false; // Swallow the error and report that the file is out of date
}
}
@ -118,7 +118,7 @@ bool ClusterConnectionFile::writeFile() {
return true;
} catch( Error &e ) {
TraceEvent(SevWarnAlways, "UnableToChangeConnectionFile").detail("Filename", filename).detail("ConnStr", cs.toString()).error(e);
TraceEvent(SevWarnAlways, "UnableToChangeConnectionFile").error(e).detail("Filename", filename).detail("ConnStr", cs.toString());
}
}

View File

@ -615,7 +615,7 @@ void MultiVersionDatabase::DatabaseState::fire(const Void &unused, int& userPara
}
catch(Error &e) {
optionFailed = true;
TraceEvent(SevError, "DatabaseVersionChangeOptionError").detail("Option", option.first).detail("OptionValue", printable(option.second)).error(e);
TraceEvent(SevError, "DatabaseVersionChangeOptionError").error(e).detail("Option", option.first).detail("OptionValue", printable(option.second));
}
}
@ -811,7 +811,7 @@ void MultiVersionCluster::Connector::error(const Error& e, int& userParam) {
// TODO: is it right to abandon this connection attempt?
client->failed = true;
MultiVersionApi::api->updateSupportedVersions();
TraceEvent(SevError, "ClusterConnectionError").detail("ClientLibrary", this->client->libPath).error(e);
TraceEvent(SevError, "ClusterConnectionError").error(e).detail("ClientLibrary", this->client->libPath);
}
delref();
@ -850,7 +850,7 @@ void MultiVersionCluster::ClusterState::stateChanged() {
}
catch(Error &e) {
optionLock.leave();
TraceEvent(SevError, "ClusterVersionChangeOptionError").detail("Option", option.first).detail("OptionValue", printable(option.second)).detail("LibPath", clients[newIndex]->libPath).error(e);
TraceEvent(SevError, "ClusterVersionChangeOptionError").error(e).detail("Option", option.first).detail("OptionValue", printable(option.second)).detail("LibPath", clients[newIndex]->libPath);
connectionAttempts[newIndex]->connected = false;
clients[newIndex]->failed = true;
MultiVersionApi::api->updateSupportedVersions();
@ -910,7 +910,7 @@ void MultiVersionApi::runOnExternalClients(std::function<void(Reference<ClientIn
}
}
catch(Error &e) {
TraceEvent(SevWarnAlways, "ExternalClientFailure").detail("LibPath", c->second->libPath).error(e);
TraceEvent(SevWarnAlways, "ExternalClientFailure").error(e).detail("LibPath", c->second->libPath);
if(e.code() == error_code_external_client_already_loaded) {
c = externalClients.erase(c);
continue;
@ -1342,7 +1342,7 @@ void MultiVersionApi::loadEnvironmentVariableNetworkOptions() {
}
}
catch(Error &e) {
TraceEvent(SevError, "EnvironmentVariableNetworkOptionFailed").detail("Option", option.second.name).detail("Value", valueStr).error(e);
TraceEvent(SevError, "EnvironmentVariableNetworkOptionFailed").error(e).detail("Option", option.second.name).detail("Value", valueStr);
throw environment_variable_network_option_failed();
}
}

View File

@ -522,10 +522,10 @@ ACTOR static Future<Void> monitorClientInfo( Reference<AsyncVar<Optional<Cluster
}
} catch( Error& e ) {
TraceEvent(SevError, "MonitorClientInfoError")
.error(e)
.detail("DBName", printable(dbName))
.detail("ConnectionFile", ccf && ccf->canGetFilename() ? ccf->getFilename() : "")
.detail("ConnectionString", ccf ? ccf->getConnectionString().toString() : "")
.error(e);
.detail("ConnectionString", ccf ? ccf->getConnectionString().toString() : "");
throw;
}
@ -817,8 +817,8 @@ void setNetworkOption(FDBNetworkOptions::Option option, Optional<StringRef> valu
tlsOptions->set_verify_peers({ value.get().toString() });
} catch( Error& e ) {
TraceEvent(SevWarnAlways, "TLSValidationSetError")
.detail("Input", value.get().toString() )
.error( e );
.error( e )
.detail("Input", value.get().toString() );
throw invalid_option_value();
}
break;
@ -1269,12 +1269,10 @@ ACTOR Future<Key> getKey( Database cx, KeySelector k, Future<Version> version, T
Void _ = wait(delay(CLIENT_KNOBS->WRONG_SHARD_SERVER_DELAY, info.taskID));
} else {
if(e.code() != error_code_actor_cancelled) {
TraceEvent(SevInfo, "GetKeyError")
.error(e)
.detail("AtKey", printable(k.getKey()))
.detail("Offset", k.offset);
}
TraceEvent(SevInfo, "GetKeyError")
.error(e)
.detail("AtKey", printable(k.getKey()))
.detail("Offset", k.offset);
throw e;
}
}
@ -2483,11 +2481,11 @@ Future<Void> Transaction::commitMutations() {
size_t transactionSize = tr.transaction.mutations.expectedSize() + tr.transaction.read_conflict_ranges.expectedSize() + tr.transaction.write_conflict_ranges.expectedSize();
if (transactionSize > (uint64_t)FLOW_KNOBS->PACKET_WARNING) {
TraceEvent(!g_network->isSimulated() ? SevWarnAlways : SevWarn, "LargeTransaction")
.suppressFor(1.0)
.detail("Size", transactionSize)
.detail("NumMutations", tr.transaction.mutations.size())
.detail("ReadConflictSize", tr.transaction.read_conflict_ranges.expectedSize())
.detail("WriteConflictSize", tr.transaction.write_conflict_ranges.expectedSize())
.suppressFor(1.0);
.detail("WriteConflictSize", tr.transaction.write_conflict_ranges.expectedSize());
}
if(!apiVersionAtLeast(300)) {
@ -2693,7 +2691,7 @@ ACTOR Future<GetReadVersionReply> getConsistentReadVersion( DatabaseContext *cx,
}
}
} catch (Error& e) {
if( e.code() != error_code_broken_promise && e.code() != error_code_actor_cancelled )
if( e.code() != error_code_broken_promise )
TraceEvent(SevError, "GetConsistentReadVersionError").error(e);
throw;
}

View File

@ -1905,12 +1905,11 @@ void ReadYourWritesTransaction::debugLogRetries(Optional<Error> error) {
fprintf(stderr, "fdb WARNING: long transaction (%.2fs elapsed%s, %d retries, %s)\n", elapsed, transactionNameStr.c_str(), retries, committed ? "committed" : error.get().what());
{
TraceEvent trace = TraceEvent("LongTransaction");
if(!transactionDebugInfo->transactionName.empty())
trace.detail("TransactionName", printable(StringRef(transactionDebugInfo->transactionName)));
trace.detail("Elapsed", elapsed).detail("Retries", retries).detail("Committed", committed);
if(error.present())
trace.error(error.get(), true);
if(!transactionDebugInfo->transactionName.empty())
trace.detail("TransactionName", printable(StringRef(transactionDebugInfo->transactionName)));
trace.detail("Elapsed", elapsed).detail("Retries", retries).detail("Committed", committed);
}
transactionDebugInfo->lastRetryLogTime = now();
}

View File

@ -390,18 +390,18 @@ public:
}
} catch(Error &e) {
TraceEvent(SevWarn, "TB_ExecuteFailure")
.error(e)
.detail("TaskUID", task->key.printable())
.detail("TaskType", task->params[Task::reservedTaskParamKeyType].printable())
.detail("Priority", task->getPriority())
.error(e);
.detail("Priority", task->getPriority());
try {
Void _ = wait(taskFunc->handleError(cx, task, e));
} catch(Error &e) {
TraceEvent(SevWarn, "TB_ExecuteFailureLogErrorFailed")
.error(e) // output handleError() error instead of original task error
.detail("TaskUID", task->key.printable())
.detail("TaskType", task->params[Task::reservedTaskParamKeyType].printable())
.detail("Priority", task->getPriority())
.error(e); // output handleError() error instead of original task error
.detail("Priority", task->getPriority());
}
}

View File

@ -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).suppressFor(1.0);
TraceEvent("AsyncFileOpened").suppressFor(1.0).detail("Filename", filename).detail("Fd", r->result).detail("Flags", flags);
if ((flags & OPEN_LOCK) && !lock_fd(r->result)) {
TraceEvent(SevError, "UnableToLockFile").detail("Filename", filename).GetLastError();
@ -254,7 +254,7 @@ private:
static void error( const char* context, int fd, eio_req* r, Reference<ErrorInfo> const& err = Reference<ErrorInfo>() ) {
Error e = io_error();
errno = r->errorno;
TraceEvent(context).detail("Fd", fd).detail("Result", r->result).GetLastError().error(e);
TraceEvent(context).error(e).detail("Fd", fd).detail("Result", r->result).GetLastError();
if (err) err->set(e);
else throw e;
}
@ -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).suppressFor(1.0);
TraceEvent("AsyncFileClosed").suppressFor(1.0).detail("Fd", fd);
}
ACTOR static Future<int> read_impl( int fd, void* data, int length, int64_t offset ) {

View File

@ -112,8 +112,8 @@ public:
Error e = errno==ENOENT ? file_not_found() : io_error();
int ecode = errno; // Save errno in case it is modified before it is used below
TraceEvent ev("AsyncFileKAIOOpenFailed");
ev.detail("Filename", filename).detailf("Flags", "%x", flags)
.detailf("OSFlags", "%x", openFlags(flags) | O_DIRECT).detailf("Mode", "0%o", mode).error(e).GetLastError();
ev.error(e).detail("Filename", filename).detailf("Flags", "%x", flags)
.detailf("OSFlags", "%x", openFlags(flags) | O_DIRECT).detailf("Mode", "0%o", mode).GetLastError();
if(ecode == EINVAL)
ev.detail("Description", "Invalid argument - Does the target filesystem support KAIO?");
return e;

View File

@ -238,7 +238,7 @@ public:
state Error err = e;
std::string currentFilename = ( wrappedFile.isReady() && !wrappedFile.isError() ) ? wrappedFile.get()->getFilename() : actualFilename;
currentProcess->machine->openFiles.erase( currentFilename );
//TraceEvent("AsyncFileNonDurableOpenError").detail("Filename", filename).detail("Address", currentProcess->address).error(e, true).detail("Addr", g_simulator.getCurrentProcess()->address);
//TraceEvent("AsyncFileNonDurableOpenError").error(e, true).detail("Filename", filename).detail("Address", currentProcess->address).detail("Addr", g_simulator.getCurrentProcess()->address);
Void _ = wait( g_simulator.onProcess( currentProcess, currentTaskID ) );
throw err;
}

View File

@ -85,7 +85,7 @@ public:
static void onReadReady( Promise<int> onReady, const boost::system::error_code& error, size_t bytesRead ) {
if (error) {
Error e = io_error();
TraceEvent("AsyncReadError").GetLastError().error(e)
TraceEvent("AsyncReadError").error(e).GetLastError()
.detail("ASIOCode", error.value())
.detail("ASIOMessage", error.message());
onReady.sendError(e);
@ -96,7 +96,7 @@ public:
static void onWriteReady( Promise<Void> onReady, size_t bytesExpected, const boost::system::error_code& error, size_t bytesWritten ) {
if (error) {
Error e = io_error();
TraceEvent("AsyncWriteError").GetLastError().error(e)
TraceEvent("AsyncWriteError").error(e).GetLastError()
.detail("ASIOCode", error.value())
.detail("ASIOMessage", error.message());
onReady.sendError(e);
@ -174,4 +174,4 @@ private:
#endif
#endif
#endif

View File

@ -134,12 +134,12 @@ private:
if(history.checksum != 0 && history.checksum != checksum) {
// For reads, verify the stored sum if it is not 0. If it fails, clear it.
TraceEvent (SevError, "AsyncFileLostWriteDetected")
.error(checksum_failed())
.detail("Filename", m_f->getFilename())
.detail("PageNumber", page)
.detail("ChecksumOfPage", checksum)
.detail("ChecksumHistory", history.checksum)
.detail("LastWriteTime", history.timestamp)
.error(checksum_failed());
.detail("LastWriteTime", history.timestamp);
history.checksum = 0;
}
}

View File

@ -182,7 +182,7 @@ Reference<BlobStoreEndpoint> BlobStoreEndpoint::fromString(std::string const &ur
} catch(std::string &err) {
if(error != nullptr)
*error = err;
TraceEvent(SevWarnAlways, "BlobStoreEndpointBadURL").detail("Description", err).detail("Format", getURLFormat()).detail("URL", url).suppressFor(60, true);
TraceEvent(SevWarnAlways, "BlobStoreEndpointBadURL").suppressFor(60).detail("Description", err).detail("Format", getURLFormat()).detail("URL", url);
throw backup_invalid_url();
}
}
@ -341,11 +341,11 @@ ACTOR Future<Optional<json_spirit::mObject>> tryReadJSONFile(std::string path) {
if(json.type() == json_spirit::obj_type)
return json.get_obj();
else
TraceEvent(SevWarn, "BlobCredentialFileNotJSONObject").detail("File", path).suppressFor(60, true);
TraceEvent(SevWarn, "BlobCredentialFileNotJSONObject").suppressFor(60).detail("File", path);
} catch(Error &e) {
if(e.code() != error_code_actor_cancelled)
TraceEvent(SevWarn, errorEventType).detail("File", path).error(e).suppressFor(60, true);
TraceEvent(SevWarn, errorEventType).error(e).suppressFor(60).detail("File", path);
}
return Optional<json_spirit::mObject>();
@ -408,10 +408,9 @@ ACTOR Future<BlobStoreEndpoint::ReusableConnection> connect_impl(Reference<BlobS
// If the connection expires in the future then return it
if(rconn.expirationTime > now()) {
TraceEvent("BlobStoreEndpointReusingConnected")
TraceEvent("BlobStoreEndpointReusingConnected").suppressFor(60)
.detail("RemoteEndpoint", rconn.conn->getPeerAddress())
.detail("ExpiresIn", rconn.expirationTime - now())
.suppressFor(60, true);
.detail("ExpiresIn", rconn.expirationTime - now());
return rconn;
}
}
@ -420,10 +419,9 @@ ACTOR Future<BlobStoreEndpoint::ReusableConnection> connect_impl(Reference<BlobS
service = b->knobs.secure_connection ? "https" : "http";
state Reference<IConnection> conn = wait(INetworkConnections::net()->connect(b->host, service, b->knobs.secure_connection ? true : false));
TraceEvent("BlobStoreEndpointNewConnection")
TraceEvent("BlobStoreEndpointNewConnection").suppressFor(60)
.detail("RemoteEndpoint", conn->getPeerAddress())
.detail("ExpiresIn", b->knobs.max_connection_life)
.suppressFor(60, true);
.detail("ExpiresIn", b->knobs.max_connection_life);
if(b->lookupSecret)
Void _ = wait(b->updateSecret());
@ -521,6 +519,16 @@ ACTOR Future<Reference<HTTP::Response>> doRequest_impl(Reference<BlobStoreEndpoi
retryable = retryable && (thisTry < maxTries);
TraceEvent event(SevWarn, retryable ? "BlobStoreEndpointRequestFailedRetryable" : "BlobStoreEndpointRequestFailed");
// Attach err to trace event if present, otherwise extract some stuff from the response
if(err.present()) {
event.error(err.get());
}
event.suppressFor(60);
if(!err.present()) {
event.detail("ResponseCode", r->code);
}
event.detail("ConnectionEstablished", connectionEstablished);
if(remoteAddress.present())
@ -530,8 +538,7 @@ ACTOR Future<Reference<HTTP::Response>> doRequest_impl(Reference<BlobStoreEndpoi
event.detail("Verb", verb)
.detail("Resource", resource)
.detail("ThisTry", thisTry)
.suppressFor(60, true);
.detail("ThisTry", thisTry);
// If r is not valid or not code 429 then increment the try count. 429's will not count against the attempt limit.
if(!r || r->code != 429)
@ -542,13 +549,6 @@ ACTOR Future<Reference<HTTP::Response>> doRequest_impl(Reference<BlobStoreEndpoi
// Double but limit the *next* nextRetryDelay.
nextRetryDelay = std::min(nextRetryDelay * 2, 60.0);
// Attach err to trace event if present, otherwise extract some stuff from the response
if(err.present())
event.error(err.get());
else {
event.detail("ResponseCode", r->code);
}
if(retryable) {
// If r is valid then obey the Retry-After response header if present.
if(r) {
@ -702,13 +702,13 @@ ACTOR Future<Void> listBucketStream_impl(Reference<BlobStoreEndpoint> bstore, st
lastFile = result.commonPrefixes.back();
if(lastFile.empty()) {
TraceEvent(SevWarn, "BlobStoreEndpointListNoNextMarker").detail("Resource", fullResource).suppressFor(60, true);
TraceEvent(SevWarn, "BlobStoreEndpointListNoNextMarker").suppressFor(60).detail("Resource", fullResource);
throw backup_error();
}
}
} catch(Error &e) {
if(e.code() != error_code_actor_cancelled)
TraceEvent(SevWarn, "BlobStoreEndpointListResultParseError").detail("Resource", fullResource).error(e).suppressFor(60, true);
TraceEvent(SevWarn, "BlobStoreEndpointListResultParseError").error(e).suppressFor(60).detail("Resource", fullResource);
throw http_bad_response();
}
}

View File

@ -92,7 +92,7 @@ void SimpleFailureMonitor::setStatus( NetworkAddress const& address, FailureStat
void SimpleFailureMonitor::endpointNotFound( Endpoint const& endpoint ) {
// SOMEDAY: Expiration (this "leaks" memory)
TraceEvent("EndpointNotFound").detail("Address", endpoint.address).detail("Token", endpoint.token).suppressFor(1.0);
TraceEvent("EndpointNotFound").suppressFor(1.0).detail("Address", endpoint.address).detail("Token", endpoint.token);
endpointKnownFailed.set( endpoint, true );
}

View File

@ -376,14 +376,14 @@ struct Peer : NonCopyable {
Void _ = wait( delayJittered( std::max(0.0, self->lastConnectTime+self->reconnectionDelay - now()) ) ); // Don't connect() to the same peer more than once per 2 sec
self->lastConnectTime = now();
TraceEvent("ConnectingTo", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).suppressFor(1.0);
TraceEvent("ConnectingTo", conn ? conn->getDebugID() : UID()).suppressFor(1.0).detail("PeerAddr", self->destination);
Reference<IConnection> _conn = wait( timeout( INetworkConnections::net()->connect(self->destination), FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT, Reference<IConnection>() ) );
if (_conn) {
conn = _conn;
TraceEvent("ConnectionExchangingConnectPacket", conn->getDebugID()).detail("PeerAddr", self->destination).suppressFor(1.0);
TraceEvent("ConnectionExchangingConnectPacket", conn->getDebugID()).suppressFor(1.0).detail("PeerAddr", self->destination);
self->prependConnectPacket();
} else {
TraceEvent("ConnectionTimedOut", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).suppressFor(1.0);
TraceEvent("ConnectionTimedOut", conn ? conn->getDebugID() : UID()).suppressFor(1.0).detail("PeerAddr", self->destination);
throw connection_failed();
}
@ -415,10 +415,10 @@ struct Peer : NonCopyable {
bool ok = e.code() == error_code_connection_failed || e.code() == error_code_actor_cancelled || ( g_network->isSimulated() && e.code() == error_code_checksum_failed );
if(self->compatible) {
TraceEvent(ok ? SevInfo : SevWarnAlways, "ConnectionClosed", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).error(e, true).suppressFor(1.0);
TraceEvent(ok ? SevInfo : SevWarnAlways, "ConnectionClosed", conn ? conn->getDebugID() : UID()).error(e, true).suppressFor(1.0).detail("PeerAddr", self->destination);
}
else {
TraceEvent(ok ? SevInfo : SevWarnAlways, "IncompatibleConnectionClosed", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).error(e, true);
TraceEvent(ok ? SevInfo : SevWarnAlways, "IncompatibleConnectionClosed", conn ? conn->getDebugID() : UID()).error(e, true).detail("PeerAddr", self->destination);
}
if (conn) {
@ -430,7 +430,7 @@ struct Peer : NonCopyable {
// Try to recover, even from serious errors, by retrying
if(self->peerReferences <= 0 && self->reliable.empty() && self->unsent.empty()) {
TraceEvent("PeerDestroy").detail("PeerAddr", self->destination).error(e).suppressFor(1.0);
TraceEvent("PeerDestroy").error(e).suppressFor(1.0).detail("PeerAddr", self->destination);
self->connect.cancel();
self->transport->peers.erase(self->destination);
delete self;
@ -559,10 +559,10 @@ static void scanPackets( TransportData* transport, uint8_t*& unprocessed_begin,
if (packetLen > FLOW_KNOBS->PACKET_WARNING) {
TraceEvent(transport->warnAlwaysForLargePacket ? SevWarnAlways : SevWarn, "Net2_LargePacket")
.suppressFor(1.0)
.detail("FromPeer", peerAddress.toString())
.detail("Length", (int)packetLen)
.detail("Token", token)
.suppressFor(1.0);
.detail("Token", token);
if(g_network->isSimulated())
transport->warnAlwaysForLargePacket = false;
@ -663,8 +663,9 @@ ACTOR static Future<Void> connectionReader(
else {
compatible = true;
TraceEvent("ConnectionEstablished", conn->getDebugID())
.suppressFor(1.0)
.detail("Peer", conn->getPeerAddress())
.detail("ConnectionId", connectionId).suppressFor(1.0);
.detail("ConnectionId", connectionId);
}
if(connectionId > 1) {
@ -676,7 +677,7 @@ ACTOR static Future<Void> connectionReader(
peerProtocolVersion = p->protocolVersion;
if (peer != nullptr) {
// Outgoing connection; port information should be what we expect
TraceEvent("ConnectedOutgoing").detail("PeerAddr", NetworkAddress( p->canonicalRemoteIp, p->canonicalRemotePort ) ).suppressFor(1.0);
TraceEvent("ConnectedOutgoing").suppressFor(1.0).detail("PeerAddr", NetworkAddress( p->canonicalRemoteIp, p->canonicalRemotePort ) );
peer->compatible = compatible;
peer->incompatibleProtocolVersionNewer = incompatibleProtocolVersionNewer;
if (!compatible) {
@ -743,7 +744,7 @@ ACTOR static Future<Void> connectionIncoming( TransportData* self, Reference<ICo
}
return Void();
} catch (Error& e) {
TraceEvent("IncomingConnectionError", conn->getDebugID()).error(e).detail("FromAddress", conn->getPeerAddress()).suppressFor(1.0);
TraceEvent("IncomingConnectionError", conn->getDebugID()).error(e).suppressFor(1.0).detail("FromAddress", conn->getPeerAddress());
conn->close();
return Void();
}
@ -755,7 +756,7 @@ ACTOR static Future<Void> listen( TransportData* self, NetworkAddress listenAddr
try {
loop {
Reference<IConnection> conn = wait( listener->accept() );
TraceEvent("ConnectionFrom", conn->getDebugID()).detail("FromAddress", conn->getPeerAddress()).suppressFor(1.0);
TraceEvent("ConnectionFrom", conn->getDebugID()).suppressFor(1.0).detail("FromAddress", conn->getPeerAddress());
incoming.add( connectionIncoming(self, conn) );
}
} catch (Error& e) {
@ -969,11 +970,11 @@ static PacketID sendPacket( TransportData* self, ISerializeSource const& what, c
}
else if (len > FLOW_KNOBS->PACKET_WARNING) {
TraceEvent(self->warnAlwaysForLargePacket ? SevWarnAlways : SevWarn, "Net2_LargePacket")
.suppressFor(1.0)
.detail("ToPeer", destination.address)
.detail("Length", (int)len)
.detail("Token", destination.token)
.backtrace()
.suppressFor(1.0);
.backtrace();
if(g_network->isSimulated())
self->warnAlwaysForLargePacket = false;

View File

@ -72,7 +72,7 @@ Future<Void> batcher(PromiseStream<std::pair<std::vector<X>, int> > out, FutureS
// Drop requests if memory is under severe pressure
if (*commitBatchesMemBytesCount + bytes > commitBatchesMemBytesLimit) {
x.reply.sendError(proxy_memory_limit_exceeded());
TraceEvent(SevWarnAlways, "ProxyCommitBatchMemoryThresholdExceeded").detail("CommitBatchesMemBytesCount", *commitBatchesMemBytesCount).detail("CommitBatchesMemLimit", commitBatchesMemBytesLimit).suppressFor(60, true);
TraceEvent(SevWarnAlways, "ProxyCommitBatchMemoryThresholdExceeded").suppressFor(60).detail("CommitBatchesMemBytesCount", *commitBatchesMemBytesCount).detail("CommitBatchesMemLimit", commitBatchesMemBytesLimit);
continue;
}

View File

@ -1194,12 +1194,12 @@ public:
TEST( kt == InjectFaults ); // Simulated machine was killed with faults
if (kt == KillInstantly) {
TraceEvent(SevWarn, "FailMachine", machine->locality.zoneId()).detail("Name", machine->name).detail("Address", machine->address).detailext("ZoneId", machine->locality.zoneId()).detail("Process", describe(*machine)).detail("Rebooting", machine->rebooting).detail("Protected", protectedAddresses.count(machine->address)).backtrace();
TraceEvent(SevWarn, "FailMachine").detail("Name", machine->name).detail("Address", machine->address).detailext("ZoneId", machine->locality.zoneId()).detail("Process", describe(*machine)).detail("Rebooting", machine->rebooting).detail("Protected", protectedAddresses.count(machine->address)).backtrace();
// This will remove all the "tracked" messages that came from the machine being killed
latestEventCache.clear();
machine->failed = true;
} else if (kt == InjectFaults) {
TraceEvent(SevWarn, "FaultMachine", machine->locality.zoneId()).detail("Name", machine->name).detail("Address", machine->address).detailext("ZoneId", machine->locality.zoneId()).detail("Process", describe(*machine)).detail("Rebooting", machine->rebooting).detail("Protected", protectedAddresses.count(machine->address)).backtrace();
TraceEvent(SevWarn, "FaultMachine").detail("Name", machine->name).detail("Address", machine->address).detailext("ZoneId", machine->locality.zoneId()).detail("Process", describe(*machine)).detail("Rebooting", machine->rebooting).detail("Protected", protectedAddresses.count(machine->address)).backtrace();
should_inject_fault = simulator_should_inject_fault;
machine->fault_injection_r = g_random->randomUniqueID().first();
machine->fault_injection_p1 = 0.1;
@ -1255,7 +1255,7 @@ public:
TEST(kt == InjectFaults); // Trying to kill by injecting faults
if(speedUpSimulation && !forceKill) {
TraceEvent(SevWarn, "AbortedKill", zoneId).detailext("ZoneId", zoneId).detail("Reason", "Unforced kill within speedy simulation.").backtrace();
TraceEvent(SevWarn, "AbortedKill").detailext("ZoneId", zoneId).detail("Reason", "Unforced kill within speedy simulation.").backtrace();
if (ktFinal) *ktFinal = None;
return false;
}
@ -1273,7 +1273,7 @@ public:
// Do nothing, if no processes to kill
if (processesOnMachine == 0) {
TraceEvent(SevWarn, "AbortedKill", zoneId).detailext("ZoneId", zoneId).detail("Reason", "The target had no processes running.").detail("Processes", processesOnMachine).detail("ProcessesPerMachine", processesPerMachine).backtrace();
TraceEvent(SevWarn, "AbortedKill").detailext("ZoneId", zoneId).detail("Reason", "The target had no processes running.").detail("Processes", processesOnMachine).detail("ProcessesPerMachine", processesPerMachine).backtrace();
if (ktFinal) *ktFinal = None;
return false;
}
@ -1310,24 +1310,24 @@ public:
}
}
if (!canKillProcesses(processesLeft, processesDead, kt, &kt)) {
TraceEvent("ChangedKillMachine", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("OrigKillType", ktOrig).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("TotalProcesses", machines.size()).detail("ProcessesPerMachine", processesPerMachine).detail("Protected", protectedWorker).detail("Unavailable", unavailable).detail("Excluded", excluded).detail("Cleared", cleared).detail("ProtectedTotal", protectedAddresses.size()).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
TraceEvent("ChangedKillMachine").detailext("ZoneId", zoneId).detail("KillType", kt).detail("OrigKillType", ktOrig).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("TotalProcesses", machines.size()).detail("ProcessesPerMachine", processesPerMachine).detail("Protected", protectedWorker).detail("Unavailable", unavailable).detail("Excluded", excluded).detail("Cleared", cleared).detail("ProtectedTotal", protectedAddresses.size()).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
}
else if ((kt == KillInstantly) || (kt == InjectFaults)) {
TraceEvent("DeadMachine", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("TotalProcesses", machines.size()).detail("ProcessesPerMachine", processesPerMachine).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
TraceEvent("DeadMachine").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("TotalProcesses", machines.size()).detail("ProcessesPerMachine", processesPerMachine).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
for (auto process : processesLeft) {
TraceEvent("DeadMachineSurvivors", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("SurvivingProcess", describe(*process));
TraceEvent("DeadMachineSurvivors").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("SurvivingProcess", describe(*process));
}
for (auto process : processesDead) {
TraceEvent("DeadMachineVictims", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("VictimProcess", describe(*process));
TraceEvent("DeadMachineVictims").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("VictimProcess", describe(*process));
}
}
else {
TraceEvent("ClearMachine", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("TotalProcesses", machines.size()).detail("ProcessesPerMachine", processesPerMachine).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
TraceEvent("ClearMachine").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("TotalProcesses", machines.size()).detail("ProcessesPerMachine", processesPerMachine).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
for (auto process : processesLeft) {
TraceEvent("ClearMachineSurvivors", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("SurvivingProcess", describe(*process));
TraceEvent("ClearMachineSurvivors").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("SurvivingProcess", describe(*process));
}
for (auto process : processesDead) {
TraceEvent("ClearMachineVictims", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("VictimProcess", describe(*process));
TraceEvent("ClearMachineVictims").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("VictimProcess", describe(*process));
}
}
}
@ -1337,7 +1337,7 @@ public:
// Check if any processes on machine are rebooting
if( processesOnMachine != processesPerMachine && kt >= RebootAndDelete ) {
TEST(true); //Attempted reboot, but the target did not have all of its processes running
TraceEvent(SevWarn, "AbortedKill", zoneId).detail("KillType", kt).detailext("ZoneId", zoneId).detail("Reason", "Machine processes does not match number of processes per machine").detail("Processes", processesOnMachine).detail("ProcessesPerMachine", processesPerMachine).backtrace();
TraceEvent(SevWarn, "AbortedKill").detail("KillType", kt).detailext("ZoneId", zoneId).detail("Reason", "Machine processes does not match number of processes per machine").detail("Processes", processesOnMachine).detail("ProcessesPerMachine", processesPerMachine).backtrace();
if (ktFinal) *ktFinal = None;
return false;
}
@ -1345,24 +1345,24 @@ public:
// Check if any processes on machine are rebooting
if ( processesOnMachine != processesPerMachine ) {
TEST(true); //Attempted reboot, but the target did not have all of its processes running
TraceEvent(SevWarn, "AbortedKill", zoneId).detail("KillType", kt).detailext("ZoneId", zoneId).detail("Reason", "Machine processes does not match number of processes per machine").detail("Processes", processesOnMachine).detail("ProcessesPerMachine", processesPerMachine).backtrace();
TraceEvent(SevWarn, "AbortedKill").detail("KillType", kt).detailext("ZoneId", zoneId).detail("Reason", "Machine processes does not match number of processes per machine").detail("Processes", processesOnMachine).detail("ProcessesPerMachine", processesPerMachine).backtrace();
if (ktFinal) *ktFinal = None;
return false;
}
TraceEvent("KillMachine", zoneId).detailext("ZoneId", zoneId).detail("Kt", kt).detail("KtOrig", ktOrig).detail("KillableMachines", processesOnMachine).detail("ProcessPerMachine", processesPerMachine).detail("KillChanged", kt!=ktOrig);
TraceEvent("KillMachine").detailext("ZoneId", zoneId).detail("Kt", kt).detail("KtOrig", ktOrig).detail("KillableMachines", processesOnMachine).detail("ProcessPerMachine", processesPerMachine).detail("KillChanged", kt!=ktOrig);
if ( kt < RebootAndDelete ) {
if(kt == InjectFaults && machines[zoneId].machineProcess != nullptr)
killProcess_internal( machines[zoneId].machineProcess, kt );
for (auto& process : machines[zoneId].processes) {
TraceEvent("KillMachineProcess", zoneId).detail("KillType", kt).detail("Process", process->toString()).detail("StartingClass", process->startingClass.toString()).detail("Failed", process->failed).detail("Excluded", process->excluded).detail("Cleared", process->cleared).detail("Rebooting", process->rebooting);
TraceEvent("KillMachineProcess").detail("KillType", kt).detail("Process", process->toString()).detail("StartingClass", process->startingClass.toString()).detail("Failed", process->failed).detail("Excluded", process->excluded).detail("Cleared", process->cleared).detail("Rebooting", process->rebooting);
if (process->startingClass != ProcessClass::TesterClass)
killProcess_internal( process, kt );
}
}
else if ( kt == Reboot || kt == RebootAndDelete ) {
for (auto& process : machines[zoneId].processes) {
TraceEvent("KillMachineProcess", zoneId).detail("KillType", kt).detail("Process", process->toString()).detail("StartingClass", process->startingClass.toString()).detail("Failed", process->failed).detail("Excluded", process->excluded).detail("Cleared", process->cleared).detail("Rebooting", process->rebooting);
TraceEvent("KillMachineProcess").detail("KillType", kt).detail("Process", process->toString()).detail("StartingClass", process->startingClass.toString()).detail("Failed", process->failed).detail("Excluded", process->excluded).detail("Cleared", process->cleared).detail("Rebooting", process->rebooting);
if (process->startingClass != ProcessClass::TesterClass)
doReboot(process, kt );
}
@ -1422,11 +1422,11 @@ public:
TraceEvent("DeadDataCenter").detailext("DataCenter", dcId).detail("KillType", kt).detail("DcZones", datacenterZones.size()).detail("DcProcesses", dcProcesses).detail("ProcessesDead", processesDead.size()).detail("ProcessesLeft", processesLeft.size()).detail("TLogPolicy", tLogPolicy->info()).detail("StoragePolicy", storagePolicy->info());
for (auto process : processesLeft) {
auto zoneId = process->locality.zoneId();
TraceEvent("DeadDcSurvivors", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("SurvivingProcess", describe(*process));
TraceEvent("DeadDcSurvivors").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("SurvivingProcess", describe(*process));
}
for (auto process : processesDead) {
auto zoneId = process->locality.zoneId();
TraceEvent("DeadDcVictims", zoneId).detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("VictimProcess", describe(*process));
TraceEvent("DeadDcVictims").detailext("ZoneId", zoneId).detail("KillType", kt).detail("ProcessesLeft", processesLeft.size()).detail("ProcessesDead", processesDead.size()).detail("VictimProcess", describe(*process));
}
}
}

View File

@ -1430,7 +1430,7 @@ ACTOR Future<Void> failureDetectionServer( UID uniqueID, FutureStream< FailureMo
for(auto it=currentStatus.begin(); it!=currentStatus.end(); it++)
if (it->second.penultimateRequestTime) {
delays.push_back(it->second.latency(t));
TraceEvent("FDData", uniqueID).detail("S", it->first.toString()).detail("L", it->second.latency(t));
//TraceEvent("FDData", uniqueID).detail("S", it->first.toString()).detail("L", it->second.latency(t));
}
int pivot = std::max(0, (int)delays.size()-2);
double pivotDelay = 0;
@ -1440,7 +1440,7 @@ ACTOR Future<Void> failureDetectionServer( UID uniqueID, FutureStream< FailureMo
}
pivotDelay = std::max(0.0, pivotDelay - FLOW_KNOBS->SERVER_REQUEST_INTERVAL);
TraceEvent("FailureDetectionPoll", uniqueID).detail("PivotDelay", pivotDelay).detail("Clients", currentStatus.size());
//TraceEvent("FailureDetectionPoll", uniqueID).detail("PivotDelay", pivotDelay).detail("Clients", currentStatus.size());
//TraceEvent("FailureDetectionAcceptableDelay").detail("Delay", acceptableDelay1000);
for(auto it = currentStatus.begin(); it != currentStatus.end(); ) {

View File

@ -926,7 +926,7 @@ ACTOR Future<Void> dataDistributionRelocator( DDQueueData *self, RelocateData rd
}
TEST(true); //did not find a healthy destination team on the first attempt
stuckCount++;
TraceEvent(stuckCount > 50 ? SevWarnAlways : SevWarn, "BestTeamStuck", masterId).detail("Count", stuckCount).suppressFor(1.0);
TraceEvent(stuckCount > 50 ? SevWarnAlways : SevWarn, "BestTeamStuck", masterId).suppressFor(1.0).detail("Count", stuckCount);
Void _ = wait( delay( SERVER_KNOBS->BEST_TEAM_STUCK_DELAY, TaskDataDistributionLaunch ) );
}

View File

@ -209,7 +209,7 @@ ACTOR Future<Void> trackShardBytes(
shardSize->set( metrics );
} catch( Error &e ) {
//TraceEvent("ShardSizeUpdateError").detail("Begin", printable(keys.begin)).detail("End", printable(keys.end)).detail("TrackerID", trackerID).error(e, true);
//TraceEvent("ShardSizeUpdateError").error(e, true).detail("Begin", printable(keys.begin)).detail("End", printable(keys.end)).detail("TrackerID", trackerID);
Void _ = wait( tr.onError(e) );
}
}

View File

@ -253,7 +253,7 @@ public:
waitfor.push_back( files[1].f->truncate( files[1].size ) );
if(fileSizeWarningLimit > 0 && files[1].size > fileSizeWarningLimit) {
TraceEvent(SevWarnAlways, "DiskQueueFileTooLarge", dbgid).detail("Filename", filename(1)).detail("Size", files[1].size).suppressFor(1.0);
TraceEvent(SevWarnAlways, "DiskQueueFileTooLarge", dbgid).suppressFor(1.0).detail("Filename", filename(1)).detail("Size", files[1].size);
}
}
}
@ -322,7 +322,7 @@ public:
delete pageMem;
TEST(true); // push error
TEST(2==syncFiles.size()); // push spanning both files error
TraceEvent(SevError, "RDQPushAndCommitError", dbgid).detail("InitialFilename0", filename).error(e, true);
TraceEvent(SevError, "RDQPushAndCommitError", dbgid).error(e, true).detail("InitialFilename0", filename);
if (errorPromise.canBeSet()) errorPromise.sendError(e);
if (pushing.canBeSet()) pushing.sendError(e);
@ -428,8 +428,8 @@ public:
.detail("File0", self->filename(0));
} catch( Error &e ) {
TraceEvent(SevError, "DiskQueueShutdownError", self->dbgid)
.detail("Reason", e.code() == error_code_platform_error ? "could not delete database" : "unknown")
.error(e,true);
.error(e,true)
.detail("Reason", e.code() == error_code_platform_error ? "could not delete database" : "unknown");
error = e;
}
@ -533,7 +533,7 @@ public:
return result.str;
} catch (Error& e) {
bool ok = e.code() == error_code_file_not_found;
TraceEvent(ok ? SevInfo : SevError, "RDQReadFirstAndLastPagesError", self->dbgid).detail("File0Name", self->files[0].dbgFilename).error(e, true);
TraceEvent(ok ? SevInfo : SevError, "RDQReadFirstAndLastPagesError", self->dbgid).error(e, true).detail("File0Name", self->files[0].dbgFilename);
if (!self->error.isSet()) self->error.sendError(e);
throw;
}
@ -588,7 +588,7 @@ public:
return result;
} catch (Error& e) {
TEST(true); // Read next page error
TraceEvent(SevError, "RDQReadNextPageError", self->dbgid).detail("File0Name", self->files[0].dbgFilename).error(e, true);
TraceEvent(SevError, "RDQReadNextPageError", self->dbgid).error(e, true).detail("File0Name", self->files[0].dbgFilename);
if (!self->error.isSet()) self->error.sendError(e);
throw;
}
@ -633,7 +633,7 @@ public:
return Void();
} catch (Error& e) {
TraceEvent(SevError, "RDQTruncateBeforeLastReadPageError", self->dbgid).detail("File0Name", self->files[0].dbgFilename).error(e);
TraceEvent(SevError, "RDQTruncateBeforeLastReadPageError", self->dbgid).error(e).detail("File0Name", self->files[0].dbgFilename);
if (!self->error.isSet()) self->error.sendError(e);
throw;
}
@ -712,11 +712,11 @@ public:
if( pushedPageCount() >= 8000 ) {
TraceEvent( warnAlwaysForMemory ? SevWarnAlways : SevWarn, "DiskQueueMemoryWarning", dbgid)
.suppressFor(1.0)
.detail("PushedPages", pushedPageCount())
.detail("NextPageSeq", nextPageSeq)
.detail("Details", format("%d pages", pushedPageCount()))
.detail("File0Name", rawQueue->files[0].dbgFilename)
.suppressFor(1.0);
.detail("File0Name", rawQueue->files[0].dbgFilename);
if(g_network->isSimulated())
warnAlwaysForMemory = false;
}

View File

@ -103,14 +103,14 @@ struct PageChecksumCodec {
if(!write && sum != *pSumInPage) {
if(!silent)
TraceEvent (SevError, "SQLitePageChecksumFailure")
.error(checksum_failed())
.detail("CodecPageSize", pageSize)
.detail("CodecReserveSize", reserveSize)
.detail("Filename", filename)
.detail("PageNumber", pageNumber)
.detail("PageSize", pageLen)
.detail("ChecksumInPage", pSumInPage->toString())
.detail("ChecksumCalculated", sum.toString())
.error(checksum_failed());
.detail("ChecksumCalculated", sum.toString());
return false;
}
@ -1832,8 +1832,8 @@ private:
}
} catch (Error& e) {
TraceEvent(SevError, "KVDoCloseError", self->logID)
.detail("Reason", e.code() == error_code_platform_error ? "could not delete database" : "unknown")
.error(e,true);
.error(e,true)
.detail("Reason", e.code() == error_code_platform_error ? "could not delete database" : "unknown");
error = e;
}

View File

@ -97,14 +97,14 @@ ACTOR Future<Void> getRate(UID myID, MasterInterface master, int64_t* inTransact
when(GetRateInfoReply rep = wait(reply)) {
reply = Never();
*outTransactionRate = rep.transactionRate;
TraceEvent("MasterProxyRate", myID).detail("Rate", rep.transactionRate).detail("Lease", rep.leaseDuration).detail("ReleasedTransactions", *inTransactionCount - lastTC);
//TraceEvent("MasterProxyRate", myID).detail("Rate", rep.transactionRate).detail("Lease", rep.leaseDuration).detail("ReleasedTransactions", *inTransactionCount - lastTC);
lastTC = *inTransactionCount;
leaseTimeout = delay(rep.leaseDuration);
nextRequestTimer = delayJittered(rep.leaseDuration / 2);
}
when(Void _ = wait(leaseTimeout)) {
*outTransactionRate = 0;
TraceEvent("MasterProxyRate", myID).detail("Rate", 0).detail("Lease", "Expired");
//TraceEvent("MasterProxyRate", myID).detail("Rate", 0).detail("Lease", "Expired");
leaseTimeout = Never();
}
}

View File

@ -314,10 +314,10 @@ ACTOR Future<Void> startMoveKeys( Database occ, KeyRange keys, vector<UID> serve
if(retries%10 == 0) {
TraceEvent(retries == 50 ? SevWarnAlways : SevWarn, "StartMoveKeysRetrying", relocationIntervalId)
.error(err)
.detail("Keys", printable(keys))
.detail("BeginKey", printable(begin))
.detail("NumTries", retries)
.error(err);
.detail("NumTries", retries);
}
}
}

View File

@ -1005,7 +1005,7 @@ namespace oldTLog {
logData->queueCommittedVersion.set(ver);
self->queueCommitEnd.set(commitNumber);
TraceEvent("TLogCommitDurable", self->dbgid).detail("Version", ver);
//TraceEvent("TLogCommitDurable", self->dbgid).detail("Version", ver);
return Void();
}

View File

@ -74,7 +74,7 @@ ACTOR Future<int64_t> getDataInFlight( Database cx, WorkerInterface masterWorker
sscanf(md.getValue("TotalBytes").c_str(), "%lld", &dataInFlight);
return dataInFlight;
} catch( Error &e ) {
TraceEvent("QuietDatabaseFailure", masterWorker.id()).detail("Reason", "Failed to extract DataInFlight").error(e);
TraceEvent("QuietDatabaseFailure", masterWorker.id()).error(e).detail("Reason", "Failed to extract DataInFlight");
throw;
}

View File

@ -229,7 +229,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
state Future<ISimulator::KillType> onShutdown = process->onShutdown();
try {
TraceEvent("SimulatedRebooterStarting", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedRebooterStarting").detail("Cycles", cycles).detail("RandomId", randomId)
.detailext("ZoneId", localities.zoneId())
.detailext("DataHall", localities.dataHallId())
.detail("Address", process->address.toString())
@ -270,10 +270,10 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
if(e.code() != error_code_actor_cancelled)
printf("SimulatedFDBDTerminated: %s\n", e.what());
ASSERT( destructed || g_simulator.getCurrentProcess() == process ); // simulatedFDBD catch called on different process
TraceEvent(e.code() == error_code_actor_cancelled || e.code() == error_code_file_not_found || destructed ? SevInfo : SevError, "SimulatedFDBDTerminated", localities.zoneId()).error(e, true);
TraceEvent(e.code() == error_code_actor_cancelled || e.code() == error_code_file_not_found || destructed ? SevInfo : SevError, "SimulatedFDBDTerminated").error(e, true).detailext("ZoneId", localities.zoneId());
}
TraceEvent("SimulatedFDBDDone", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedFDBDDone").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", process->address)
.detail("Excluded", process->excluded)
.detailext("ZoneId", localities.zoneId())
@ -282,7 +282,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
if (!onShutdown.isReady())
onShutdown = ISimulator::InjectFaults;
} catch (Error& e) {
TraceEvent(destructed ? SevInfo : SevError, "SimulatedFDBDRebooterError", localities.zoneId()).detail("RandomId", randomId).error(e, true);
TraceEvent(destructed ? SevInfo : SevError, "SimulatedFDBDRebooterError").error(e, true).detailext("ZoneId", localities.zoneId()).detail("RandomId", randomId);
onShutdown = e;
}
@ -292,7 +292,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
process->rebooting = true;
process->shutdownSignal.send(ISimulator::None);
}
TraceEvent("SimulatedFDBDWait", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedFDBDWait").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", process->address)
.detail("Excluded", process->excluded)
.detail("Rebooting", process->rebooting)
@ -304,14 +304,14 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
g_simulator.destroyProcess( process ); // Leak memory here; the process may be used in other parts of the simulation
auto shutdownResult = onShutdown.get();
TraceEvent("SimulatedFDBDShutdown", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedFDBDShutdown").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", process->address)
.detail("Excluded", process->excluded)
.detailext("ZoneId", localities.zoneId())
.detail("KillType", shutdownResult);
if( shutdownResult < ISimulator::RebootProcessAndDelete ) {
TraceEvent("SimulatedFDBDLowerReboot", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedFDBDLowerReboot").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", process->address)
.detail("Excluded", process->excluded)
.detailext("ZoneId", localities.zoneId())
@ -320,7 +320,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
}
if( onShutdown.get() == ISimulator::RebootProcessAndDelete ) {
TraceEvent("SimulatedFDBDRebootAndDelete", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedFDBDRebootAndDelete").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", process->address)
.detailext("ZoneId", localities.zoneId())
.detail("KillType", shutdownResult);
@ -337,7 +337,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(
}
}
else {
TraceEvent("SimulatedFDBDJustRepeat", localities.zoneId()).detail("Cycles", cycles).detail("RandomId", randomId)
TraceEvent("SimulatedFDBDJustRepeat").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", process->address)
.detailext("ZoneId", localities.zoneId())
.detail("KillType", shutdownResult);

View File

@ -1107,7 +1107,7 @@ ACTOR Future<Void> doQueueCommit( TLogData* self, Reference<LogData> logData ) {
logData->recoveryComplete.send(Void());
}
TraceEvent("TLogCommitDurable", self->dbgid).detail("Version", ver);
//TraceEvent("TLogCommitDurable", self->dbgid).detail("Version", ver);
if(logData->logSystem->get() && (!logData->isPrimary || logData->logRouterPoppedVersion < logData->logRouterPopToVersion)) {
logData->logRouterPoppedVersion = ver;
logData->logSystem->get()->pop(ver, logData->remoteTag, knownCommittedVersion, logData->locality);
@ -1204,7 +1204,7 @@ ACTOR Future<Void> tLogCommit(
if(req.debugID.present())
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.Before");
TraceEvent("TLogCommit", logData->logId).detail("Version", req.version);
//TraceEvent("TLogCommit", logData->logId).detail("Version", req.version);
commitMessages(logData, req.version, req.arena, req.messages, self->bytesInput);
logData->knownCommittedVersion = std::max(logData->knownCommittedVersion, req.knownCommittedVersion);

View File

@ -843,7 +843,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
last = to.first;
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) throw;
TraceEvent( (e.code() == error_code_broken_promise) ? SevInfo : SevError, "LogPopError", self->dbgid ).detail("Log", log->get().id()).error(e);
TraceEvent( (e.code() == error_code_broken_promise) ? SevInfo : SevError, "LogPopError", self->dbgid ).error(e).detail("Log", log->get().id());
return Void(); // Leaving outstandingPops filled in means no further pop requests to this tlog from this logSystem
}
}

View File

@ -200,10 +200,10 @@ static int asyncSync(sqlite3_file *pFile, int flags){
return SQLITE_OK;
} catch (Error& e) {
TraceEvent("VFSSyncError")
.error(e)
.detail("Filename", p->filename)
.detail("Sqlite3File", (int64_t)pFile)
.detail("IAsyncFile", (int64_t)p->file.getPtr())
.error(e);
.detail("IAsyncFile", (int64_t)p->file.getPtr());
return SQLITE_IOERR_FSYNC;
}
@ -528,7 +528,7 @@ static int asyncOpen(
.detail("Sqlite3File", DEBUG_DETERMINISM ? 0 : (int64_t)pFile)
.detail("IAsyncFile", DEBUG_DETERMINISM ? 0 : (int64_t)p->file.getPtr());*/
} catch (Error& e) {
TraceEvent("SQLiteOpenFail").detail("Filename", p->filename).error(e);
TraceEvent("SQLiteOpenFail").error(e).detail("Filename", p->filename);
p->~VFSAsyncFile();
return SQLITE_CANTOPEN;
}
@ -632,10 +632,10 @@ static int asyncFullPathname(
memcpy(zPathOut, s.c_str(), s.size()+1);
return SQLITE_OK;
} catch (Error& e) {
TraceEvent(SevError,"VFSAsyncFullPathnameError").detail("PathIn", (std::string)zPath).error(e);
TraceEvent(SevError,"VFSAsyncFullPathnameError").error(e).detail("PathIn", (std::string)zPath);
return SQLITE_IOERR;
} catch(...) {
TraceEvent(SevError,"VFSAsyncFullPathnameError").detail("PathIn", (std::string)zPath).error(unknown_error());
TraceEvent(SevError,"VFSAsyncFullPathnameError").error(unknown_error()).detail("PathIn", (std::string)zPath);
return SQLITE_IOERR;
}
}

View File

@ -1455,7 +1455,7 @@ bool changeDurableVersion( StorageServer* data, Version desiredDurableVersion )
data->durableVersion.set( nextDurableVersion );
if (checkFatalError.isReady()) checkFatalError.get();
TraceEvent("ForgotVersionsBefore", data->thisServerID).detail("Version", nextDurableVersion);
//TraceEvent("ForgotVersionsBefore", data->thisServerID).detail("Version", nextDurableVersion);
validate(data);
return nextDurableVersion == desiredDurableVersion;
@ -1898,7 +1898,7 @@ ACTOR Future<Void> fetchKeys( StorageServer *data, AddingShard* shard ) {
break;
} catch (Error& e) {
TraceEvent("FKBlockFail", data->thisServerID).detail("FKID", interval.pairID).error(e,true).suppressFor(1.0);
TraceEvent("FKBlockFail", data->thisServerID).error(e,true).suppressFor(1.0).detail("FKID", interval.pairID);
if (e.code() == error_code_transaction_too_old){
TEST(true); // A storage server has forgotten the history data we are fetching
Version lastFV = fetchVersion;
@ -2638,7 +2638,7 @@ ACTOR Future<Void> updateStorage(StorageServer* data) {
Void _ = wait( yield(TaskUpdateStorage) );
}
TraceEvent("StorageServerDurable", data->thisServerID).detail("Version", newOldestVersion);
//TraceEvent("StorageServerDurable", data->thisServerID).detail("Version", newOldestVersion);
Void _ = wait( durableDelay );
}

View File

@ -418,7 +418,7 @@ ACTOR Future<Void> runWorkloadAsync( Database cx, WorkloadInterface workIface, T
setupResult = Void();
} catch (Error& e) {
setupResult = operation_failed();
TraceEvent(SevError, "TestSetupError", workIface.id()).detail("Workload", workload->description()).error(e);
TraceEvent(SevError, "TestSetupError", workIface.id()).error(e).detail("Workload", workload->description());
if( e.code() == error_code_please_reboot || e.code() == error_code_please_reboot_delete) throw;
}
}
@ -709,8 +709,8 @@ ACTOR Future<DistributedTestResults> runWorkload( Database cx, std::vector< Test
metricsResults.push_back( metricTasks[i].get() );
else
TraceEvent(SevError, "TestFailure")
.detail("Reason", "Metrics not retrieved")
.error(metricTasks[i].getError())
.detail("Reason", "Metrics not retrieved")
.detail("From", workloads[i].metrics.getEndpoint().address);
}
}
@ -790,7 +790,7 @@ ACTOR Future<bool> runTest( Database cx, std::vector< TesterInterface > testers,
logMetrics( testResults.metrics );
} catch(Error& e) {
if( e.code() == error_code_timed_out ) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Test timed out").detail("Timeout", spec.timeout).error(e);
TraceEvent(SevError, "TestFailure").error(e).detail("Reason", "Test timed out").detail("Timeout", spec.timeout);
fprintf(stderr, "ERROR: Test timed out after %d seconds.\n", spec.timeout);
testResults.failures = testers.size();
testResults.successes = 0;
@ -805,7 +805,7 @@ ACTOR Future<bool> runTest( Database cx, std::vector< TesterInterface > testers,
try {
Void _ = wait( timeoutError( dumpDatabase( cx, "dump after " + printable(spec.title) + ".html", allKeys ), 30.0 ) );
} catch (Error& e) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Unable to dump database").error(e);
TraceEvent(SevError, "TestFailure").error(e).detail("Reason", "Unable to dump database");
ok = false;
}
@ -819,7 +819,7 @@ ACTOR Future<bool> runTest( Database cx, std::vector< TesterInterface > testers,
Void _ = wait(timeoutError(checkConsistency(cx, testers, database, quiescent, 10000.0, 18000, spec.databasePingDelay, dbInfo), 20000.0));
}
catch(Error& e) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Unable to perform consistency check").error(e);
TraceEvent(SevError, "TestFailure").error(e).detail("Reason", "Unable to perform consistency check");
ok = false;
}
}
@ -1047,7 +1047,7 @@ ACTOR Future<Void> runTests( Reference<AsyncVar<Optional<struct ClusterControlle
Void _ = wait(timeoutError(changeConfiguration(cx, testers, database, startingConfiguration), 2000.0));
}
catch(Error& e) {
TraceEvent(SevError, "TestFailure").detail("Reason", "Unable to set starting configuration").error(e);
TraceEvent(SevError, "TestFailure").error(e).detail("Reason", "Unable to set starting configuration");
}
}
@ -1057,8 +1057,7 @@ ACTOR Future<Void> runTests( Reference<AsyncVar<Optional<struct ClusterControlle
Void _ = wait( quietDatabase( cx, dbInfo, "Start") ||
( databasePingDelay == 0.0 ? Never() : testDatabaseLiveness( cx, databasePingDelay, "QuietDatabaseStart", startDelay ) ) );
} catch( Error& e ) {
if( e.code() != error_code_actor_cancelled )
TraceEvent("QuietDatabaseStartExternalError").error(e);
TraceEvent("QuietDatabaseStartExternalError").error(e);
throw;
}
}
@ -1079,8 +1078,7 @@ ACTOR Future<Void> runTests( Reference<AsyncVar<Optional<struct ClusterControlle
Void _ = wait( quietDatabase( cx, dbInfo, "End", 0, 2e6, 2e6 ) ||
( databasePingDelay == 0.0 ? Never() : testDatabaseLiveness( cx, databasePingDelay, "QuietDatabaseEnd" ) ) );
} catch( Error& e ) {
if( e.code() != error_code_actor_cancelled )
TraceEvent("QuietDatabaseEndExternalError").error(e);
TraceEvent("QuietDatabaseEndExternalError").error(e);
throw;
}
}

View File

@ -406,11 +406,11 @@ void startRole(UID roleId, UID workerId, std::string as, std::map<std::string, s
void endRole(UID id, std::string as, std::string reason, bool ok, Error e) {
{
TraceEvent ev("Role", id);
if(e.code() != invalid_error_code)
ev.error(e, true);
ev.detail("Transition", "End")
.detail("As", as)
.detail("Reason", reason);
if(e.code() != invalid_error_code)
ev.error(e, true);
ev.trackLatest( (id.shortString() + ".Role").c_str() );
}
@ -419,11 +419,10 @@ void endRole(UID id, std::string as, std::string reason, bool ok, Error e) {
std::string type = as + "Failed";
TraceEvent err(SevError, type.c_str(), id);
err.detail("Reason", reason);
if(e.code() != invalid_error_code) {
err.error(e, true);
}
err.detail("Reason", reason);
}
latestEventCache.clear( id.shortString() );

View File

@ -156,7 +156,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
Void _ = wait(backupAgent->abortBackup(cx, tag.toString()));
}
catch (Error& e) {
TraceEvent("BARW_DoBackupAbortBackupException", randomID).detail("Tag", printable(tag)).error(e);
TraceEvent("BARW_DoBackupAbortBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded)
throw;
}
@ -171,7 +171,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
Void _ = wait(backupAgent->submitBackup(cx, StringRef(backupContainer), g_random->randomInt(0, 100), tag.toString(), backupRanges, stopDifferentialDelay ? false : true));
}
catch (Error& e) {
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).detail("Tag", printable(tag)).error(e);
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}
@ -244,7 +244,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
}
}
catch (Error& e) {
TraceEvent("BARW_DoBackupDiscontinueBackupException", randomID).detail("Tag", printable(tag)).error(e);
TraceEvent("BARW_DoBackupDiscontinueBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}
@ -359,7 +359,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
extraBackup = backupAgent.submitBackup(cx, LiteralStringRef("file://simfdb/backups/"), g_random->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, true);
}
catch (Error& e) {
TraceEvent("BARW_SubmitBackup2Exception", randomID).detail("BackupTag", printable(self->backupTag)).error(e);
TraceEvent("BARW_SubmitBackup2Exception", randomID).error(e).detail("BackupTag", printable(self->backupTag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}
@ -423,7 +423,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
Void _ = wait(extraBackup);
}
catch (Error& e) {
TraceEvent("BARW_ExtraBackupException", randomID).detail("BackupTag", printable(self->backupTag)).error(e);
TraceEvent("BARW_ExtraBackupException", randomID).error(e).detail("BackupTag", printable(self->backupTag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}

View File

@ -215,7 +215,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
Void _ = wait(backupAgent->abortBackup(cx, tag));
}
catch (Error& e) {
TraceEvent("BARW_DoBackupAbortBackupException", randomID).detail("Tag", printable(tag)).error(e);
TraceEvent("BARW_DoBackupAbortBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded)
throw;
}
@ -250,7 +250,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
}
}
catch (Error &e) {
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).detail("Tag", printable(tag)).error(e);
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate) {
throw e;
}
@ -294,7 +294,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
}
}
catch (Error& e) {
TraceEvent("BARW_DoBackupDiscontinueBackupException", randomID).detail("Tag", printable(tag)).error(e);
TraceEvent("BARW_DoBackupDiscontinueBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}
@ -480,7 +480,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
extraBackup = backupAgent.submitBackup(self->extraDB, self->backupTag, self->backupRanges, true, self->extraPrefix, StringRef(), self->locked);
}
catch (Error& e) {
TraceEvent("BARW_SubmitBackup2Exception", randomID).detail("BackupTag", printable(self->backupTag)).error(e);
TraceEvent("BARW_SubmitBackup2Exception", randomID).error(e).detail("BackupTag", printable(self->backupTag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}
@ -520,7 +520,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
Void _ = wait(restoreAgent.submitBackup(cx, self->restoreTag, restoreRange, true, StringRef(), self->backupPrefix, self->locked));
}
catch (Error& e) {
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).detail("Tag", printable(self->restoreTag)).error(e);
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).error(e).detail("Tag", printable(self->restoreTag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}
@ -536,7 +536,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
Void _ = wait(extraBackup);
}
catch (Error& e) {
TraceEvent("BARW_ExtraBackupException", randomID).detail("BackupTag", printable(self->backupTag)).error(e);
TraceEvent("BARW_ExtraBackupException", randomID).error(e).detail("BackupTag", printable(self->backupTag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}

View File

@ -121,7 +121,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload {
TraceEvent("DRU_DoBackupInDifferentialMode").detail("Tag", printable(tag));
} catch (Error &e) {
TraceEvent("DRU_DoBackupSubmitBackupError").detail("Tag", printable(tag)).error(e);
TraceEvent("DRU_DoBackupSubmitBackupError").error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate) {
throw e;
}
@ -441,7 +441,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload {
Void _ = wait(restoreAgent.submitBackup(cx, self->restoreTag, restoreRanges, true, StringRef(), self->backupPrefix));
}
catch (Error& e) {
TraceEvent("DRU_RestoreSubmitBackupError").detail("Tag", printable(self->restoreTag)).error(e);
TraceEvent("DRU_RestoreSubmitBackupError").error(e).detail("Tag", printable(self->restoreTag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
throw;
}

View File

@ -49,7 +49,7 @@ Future<bool> checkRangeSimpleValueSize( Database cx, T* workload, uint64_t begin
Void _ = wait( success( first ) && success( last ) );
return first.get().present() && last.get().present();
} catch (Error& e) {
TraceEvent("CheckRangeError").detail("Begin", begin).detail("End", end).error(e);
TraceEvent("CheckRangeError").error(e).detail("Begin", begin).detail("End", end);
Void _ = wait( tr.onError(e) );
}
}

View File

@ -160,7 +160,7 @@ struct CommitBugWorkload : TestWorkload
else {
TEST(true); //Commit conflict
TraceEvent("CommitBug2Error").detail("AttemptedNum", i+1).error(e);
TraceEvent("CommitBug2Error").error(e).detail("AttemptedNum", i+1);
Void _ = wait(tr.onError(e));
}
}

View File

@ -228,7 +228,7 @@ struct ConsistencyCheckWorkload : TestWorkload
{
if(e.code() == error_code_attribute_not_found)
{
TraceEvent("ConsistencyCheck_StorageQueueSizeError").detail("Reason", "Could not read queue size").error(e);
TraceEvent("ConsistencyCheck_StorageQueueSizeError").error(e).detail("Reason", "Could not read queue size");
//This error occurs if we have undesirable servers; in that case just report the undesirable servers error
if(!hasUndesirableServers)
@ -532,7 +532,7 @@ struct ConsistencyCheckWorkload : TestWorkload
}
catch(Error& e)
{
TraceEvent("ConsistencyCheck_ErrorFetchingMetrics").detail("Begin", printable(shard.begin)).detail("End", printable(shard.end)).error(e);
TraceEvent("ConsistencyCheck_ErrorFetchingMetrics").error(e).detail("Begin", printable(shard.begin)).detail("End", printable(shard.end));
estimatedBytes.clear();
}
@ -843,8 +843,8 @@ struct ConsistencyCheckWorkload : TestWorkload
//If the data is not available and we aren't relocating this shard
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).suppressFor(1.0);
TraceEvent("ConsistencyCheck_StorageServerUnavailable").suppressFor(1.0).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);
//All shards should be available in quiscence
if(self->performQuiescentChecks)
@ -1107,7 +1107,7 @@ struct ConsistencyCheckWorkload : TestWorkload
for(itr = workers.begin(); itr != workers.end(); ++itr) {
ErrorOr<Standalone<VectorRef<UID>>> stores = wait(itr->first.diskStoreRequest.getReplyUnlessFailedFor(DiskStoreRequest(false), 2, 0));
if(stores.isError()) {
TraceEvent("ConsistencyCheck_GetDataStoreFailure").detail("Address", itr->first.address()).error(stores.getError());
TraceEvent("ConsistencyCheck_GetDataStoreFailure").error(stores.getError()).detail("Address", itr->first.address());
self->testFailure("Failed to get data stores");
return false;
}

View File

@ -130,8 +130,7 @@ struct CycleWorkload : TestWorkload {
self->totalLatency += now() - tstart;
}
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled)
TraceEvent(SevError, "CycleClient").error(e);
TraceEvent(SevError, "CycleClient").error(e);
throw;
}
}

View File

@ -87,7 +87,7 @@ struct FastTriggeredWatchesWorkload : TestWorkload {
//TraceEvent("FTWSetEnd").detail("Key", printable(key)).detail("Value", printable(value)).detail("Ver", tr.getCommittedVersion());
return tr.getCommittedVersion();
} catch( Error &e ) {
//TraceEvent("FTWSetError").detail("Key", printable(key)).detail("Value", printable(value)).error(e);
//TraceEvent("FTWSetError").error(e).detail("Key", printable(key)).detail("Value", printable(value));
Void _ = wait( tr.onError(e) );
}
}
@ -128,7 +128,7 @@ struct FastTriggeredWatchesWorkload : TestWorkload {
watchEnd = now();
first = false;
} catch( Error &e ) {
//TraceEvent("FTWWatchError").detail("Key", printable(setKey)).error(e);
//TraceEvent("FTWWatchError").error(e).detail("Key", printable(setKey));
Void _ = wait( tr.onError(e) );
}
}
@ -174,4 +174,4 @@ struct FastTriggeredWatchesWorkload : TestWorkload {
}
};
WorkloadFactory<FastTriggeredWatchesWorkload> FastTriggeredWatchesWorkloadFactory("FastTriggeredWatches");
WorkloadFactory<FastTriggeredWatchesWorkload> FastTriggeredWatchesWorkloadFactory("FastTriggeredWatches");

View File

@ -87,7 +87,7 @@ struct ExceptionContract {
for (auto i : expected) {
if (i.second == Always) {
TraceEvent evt(SevError, func.c_str());
evt.detail("Thrown", false).detail("Expected", "always").error(Error::fromUnvalidatedCode(i.first)).backtrace();
evt.error(Error::fromUnvalidatedCode(i.first)).detail("Thrown", false).detail("Expected", "always").backtrace();
if (augment)
augment(evt);
}

View File

@ -104,8 +104,7 @@ struct Increment : TestWorkload {
self->totalLatency += now() - tstart;
}
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled)
TraceEvent(SevError, "IncrementClient").error(e);
TraceEvent(SevError, "IncrementClient").error(e);
throw;
}
}

View File

@ -169,7 +169,7 @@ struct PerformanceWorkload : TestWorkload {
DistributedTestResults r = wait( runWorkload( cx, self->testers, self->dbName, spec ) );
results = r;
} catch(Error& e) {
TraceEvent("PerformanceRunError").detail("Workload", printable(self->probeWorkload)).error(e, true);
TraceEvent("PerformanceRunError").error(e, true).detail("Workload", printable(self->probeWorkload));
break;
}
PerfMetric tpsMetric = self->getNamedMetric( "Transactions/sec", results.metrics );

View File

@ -75,7 +75,7 @@ struct StorefrontWorkload : TestWorkload {
for(int c=0; c<clients.size(); c++)
if( clients[c].isError() ) {
errors++;
TraceEvent(SevError, "TestFailure").detail("Reason", "ClientError").error(clients[c].getError());
TraceEvent(SevError, "TestFailure").error(clients[c].getError()).detail("Reason", "ClientError");
}
clients.clear();
return inventoryCheck( cx->clone(), this, !errors );
@ -173,8 +173,7 @@ struct StorefrontWorkload : TestWorkload {
self->totalLatency += now() - tstart;
}
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled)
TraceEvent(SevError, "OrderingClient").error(e);
TraceEvent(SevError, "OrderingClient").error(e);
throw;
}
}

View File

@ -100,9 +100,9 @@ struct UnitTestWorkload : TestWorkload {
auto test = *t;
TraceEvent(result.code() != error_code_success ? SevError : SevInfo, "UnitTest")
.error(result, true)
.detail("Name", test->name)
.detail("File", test->file).detail("Line", test->line)
.error(result, true)
.detail("WallTime", wallTime)
.detail("FlowTime", simTime);
}

View File

@ -290,7 +290,7 @@ struct VersionStampWorkload : TestWorkload {
tr = ReadYourWritesTransaction(cx_is_primary ? cx : extraDB);
break;
} else if (err.code() == error_code_commit_unknown_result) {
//TraceEvent("VST_CommitUnknownResult").detail("Key", printable(key)).detail("VsKey", printable(versionStampKey)).error(e);
//TraceEvent("VST_CommitUnknownResult").error(e).detail("Key", printable(key)).detail("VsKey", printable(versionStampKey));
loop {
state ReadYourWritesTransaction cur_tr(cx_is_primary ? cx : extraDB);
cur_tr.setOption(FDBTransactionOptions::LOCK_AWARE);
@ -326,7 +326,7 @@ struct VersionStampWorkload : TestWorkload {
}
if (error) {
TraceEvent("VST_CommitFailed").detail("Key", printable(key)).detail("VsKey", printable(versionStampKey)).error(err);
TraceEvent("VST_CommitFailed").error(err).detail("Key", printable(key)).detail("VsKey", printable(versionStampKey));
Void _ = wait(tr.onError(err));
continue;
}

View File

@ -118,7 +118,7 @@ struct WatchesWorkload : TestWorkload {
extraLoc += 1000;
//TraceEvent("WatcherInitialSetup").detail("Watch", printable(watchKey)).detail("Ver", tr.getCommittedVersion());
} catch( Error &e ) {
//TraceEvent("WatcherInitialSetupError").detail("ExtraLoc", extraLoc).error(e);
//TraceEvent("WatcherInitialSetupError").error(e).detail("ExtraLoc", extraLoc);
Void _ = wait( tr.onError(e) );
}
}

View File

@ -40,7 +40,7 @@ extern void flushTraceFileVoid();
Error Error::fromUnvalidatedCode(int code) {
if (code < 0 || code > 30000) {
Error e = Error::fromCode(error_code_unknown_error);
TraceEvent(SevWarn, "ConvertedUnvalidatedErrorCode").detail("OriginalCode", code).error(e);
TraceEvent(SevWarn, "ConvertedUnvalidatedErrorCode").error(e).detail("OriginalCode", code);
return e;
}
else
@ -116,4 +116,4 @@ void ErrorCodeTable::addCode(int code, const char *name, const char *description
bool isAssertDisabled(int line) {
return FLOW_KNOBS->DISABLE_ASSERTS == -1 || FLOW_KNOBS->DISABLE_ASSERTS == line;
}
}

View File

@ -240,7 +240,7 @@ public:
try {
if (error) {
// Log the error...
TraceEvent(SevWarn, errContext, errID).detail("Message", error.value()).suppressFor(1.0);
TraceEvent(SevWarn, errContext, errID).suppressFor(1.0).detail("Message", error.value());
p.sendError( connection_failed() );
} else
p.send( Void() );
@ -411,15 +411,15 @@ private:
boost::system::error_code error;
socket.close(error);
if (error)
TraceEvent(SevWarn, "N2_CloseError", id).detail("Message", error.value()).suppressFor(1.0);
TraceEvent(SevWarn, "N2_CloseError", id).suppressFor(1.0).detail("Message", error.value());
}
void onReadError( const boost::system::error_code& error ) {
TraceEvent(SevWarn, "N2_ReadError", id).detail("Message", error.value()).suppressFor(1.0);
TraceEvent(SevWarn, "N2_ReadError", id).suppressFor(1.0).detail("Message", error.value());
closeSocket();
}
void onWriteError( const boost::system::error_code& error ) {
TraceEvent(SevWarn, "N2_WriteError", id).detail("Message", error.value()).suppressFor(1.0);
TraceEvent(SevWarn, "N2_WriteError", id).suppressFor(1.0).detail("Message", error.value());
closeSocket();
}
};
@ -900,11 +900,11 @@ Reference<IListener> Net2::listen( NetworkAddress localAddr ) {
x = invalid_local_address();
else
x = bind_failed();
TraceEvent("Net2ListenError").detail("Message", e.what()).error(x);
TraceEvent("Net2ListenError").error(x).detail("Message", e.what());
throw x;
} catch (std::exception const& e) {
Error x = unknown_error();
TraceEvent("Net2ListenError").detail("Message", e.what()).error(x);
TraceEvent("Net2ListenError").error(x).detail("Message", e.what());
throw x;
} catch (...) {
Error x = unknown_error();

View File

@ -1683,7 +1683,7 @@ void atomicReplace( std::string const& path, std::string const& content, bool te
INJECT_FAULT( io_error, "atomicReplace" );
}
catch(Error &e) {
TraceEvent(SevWarn, "AtomicReplace").detail("Path", path).error(e).GetLastError();
TraceEvent(SevWarn, "AtomicReplace").error(e).detail("Path", path).GetLastError();
if (f) fclose(f);
throw;
}

View File

@ -110,7 +110,6 @@ struct SuppressionMap {
TraceBatch g_traceBatch;
trace_clock_t g_trace_clock = TRACE_CLOCK_NOW;
std::set<StringRef> suppress;
IRandom* trace_random = NULL;
LatestEventCache latestEventCache;
@ -118,6 +117,7 @@ SuppressionMap suppressedEvents;
static TransientThresholdMetricSample<Standalone<StringRef>> *traceEventThrottlerCache;
static const char *TRACE_EVENT_THROTTLE_STARTING_TYPE = "TraceEventThrottle_";
static const char *TRACE_EVENT_INVALID_SUPPRESSION = "InvalidSuppression_";
static int TRACE_LOG_MAX_PREOPEN_BUFFER = 1000000;
static int TRACE_EVENT_MAX_SIZE = 4000;
@ -534,16 +534,6 @@ void openTraceFile(const NetworkAddress& na, uint64_t rollsize, uint64_t maxLogs
std::string baseName = format("%s.%03d.%03d.%03d.%03d.%d", baseOfBase.c_str(), (na.ip>>24)&0xff, (na.ip>>16)&0xff, (na.ip>>8)&0xff, na.ip&0xff, na.port);
g_traceLog.open( directory, baseName, logGroup, format("%lld", time(NULL)), rollsize, maxLogsSize, !g_network->isSimulated() ? na : Optional<NetworkAddress>());
// FIXME
suppress.insert( LiteralStringRef( "TLogCommitDurable" ) );
suppress.insert( LiteralStringRef( "StorageServerUpdate" ) );
suppress.insert( LiteralStringRef( "TLogCommit" ) );
suppress.insert( LiteralStringRef( "StorageServerDurable" ) );
suppress.insert( LiteralStringRef( "ForgotVersionsBefore" ) );
suppress.insert( LiteralStringRef( "FDData" ) );
suppress.insert( LiteralStringRef( "FailureDetectionPoll" ) );
suppress.insert( LiteralStringRef( "MasterProxyRate" ) );
uncancellable(recurring(&flushTraceFile, FLOW_KNOBS->TRACE_FLUSH_INTERVAL, TaskFlushTrace));
g_traceBatch.dump();
}
@ -560,59 +550,17 @@ bool traceFileIsOpen() {
return g_traceLog.isOpen();
}
bool TraceEvent::isEnabled( const char* type, Severity severity ) {
//if (!g_traceLog.isOpen()) return false;
if(g_network && severity < FLOW_KNOBS->MIN_TRACE_SEVERITY) return false;
StringRef s( (const uint8_t*)type, strlen(type) );
return !suppress.count(s);
TraceEvent::TraceEvent( const char* type, UID id ) : id(id), type(type), severity(SevInfo), initialized(false), enabled(true) {}
TraceEvent::TraceEvent( Severity severity, const char* type, UID id ) : id(id), type(type), severity(severity), initialized(false), enabled(true) {}
TraceEvent::TraceEvent( TraceInterval& interval, UID id ) : id(id), type(interval.type), severity(interval.severity), initialized(false), enabled(true) {
init(interval);
}
TraceEvent::TraceEvent( Severity severity, TraceInterval& interval, UID id ) : id(id), type(interval.type), severity(severity), initialized(false), enabled(true) {
init(interval);
}
TraceEvent::TraceEvent( const char* type, UID id ) : id(id) {
init(SevInfo, type);
detail("ID", id);
}
TraceEvent::TraceEvent( Severity severity, const char* type, UID id ) : id(id) {
init(severity, type);
detail("ID", id);
}
TraceEvent::TraceEvent(const char* type, const StringRef& zoneId) {
id = UID(hashlittle(zoneId.begin(), zoneId.size(), 0), 0);
init(SevInfo, type);
detailext("ID", zoneId);
}
TraceEvent::TraceEvent(Severity severity, const char* type, const StringRef& zoneId) {
id = UID(hashlittle(zoneId.begin(), zoneId.size(), 0), 0);
init(severity, type);
detailext("ID", zoneId);
}
TraceEvent::TraceEvent(const char* type, const Optional<Standalone<StringRef>>& zoneId) {
id = zoneId.present() ? UID(hashlittle(zoneId.get().begin(), zoneId.get().size(), 0), 0) : UID(-1LL,0);
init(SevInfo, type);
detailext("ID", zoneId);
}
TraceEvent::TraceEvent(Severity severity, const char* type, const Optional<Standalone<StringRef>>& zoneId) {
id = zoneId.present() ? UID(hashlittle(zoneId.get().begin(), zoneId.get().size(), 0), 0) : UID(-1LL, 0);
init(severity, type);
detailext("ID", zoneId);
}
TraceEvent::TraceEvent( TraceInterval& interval, UID id ) : id(id) {
init(interval.severity, interval);
detail("ID", id);
}
TraceEvent::TraceEvent( Severity severity, TraceInterval& interval, UID id ) : id(id) {
init(severity, interval);
detail("ID", id);
}
bool TraceEvent::init( Severity severity, TraceInterval& interval ) {
bool result = init( severity, interval.type );
bool TraceEvent::init( TraceInterval& interval ) {
bool result = init();
switch (interval.count++) {
case 0: { detail("BeginPair", interval.pairID); break; }
case 1: { detail("EndPair", interval.pairID); break; }
@ -621,13 +569,14 @@ bool TraceEvent::init( Severity severity, TraceInterval& interval ) {
return result;
}
bool TraceEvent::init( Severity severity, const char* type ) {
bool TraceEvent::init() {
if(initialized) {
return enabled;
}
initialized = true;
ASSERT(*type != '\0');
this->type = type;
this->severity = severity;
enabled = isEnabled(type, severity);
enabled = enabled && ( !g_network || severity >= FLOW_KNOBS->MIN_TRACE_SEVERITY );
// Backstop to throttle very spammy trace events
if (enabled && g_network && !g_network->isSimulated() && severity > SevDebug && isNetworkThread()) {
@ -657,6 +606,10 @@ bool TraceEvent::init( Severity severity, const char* type ) {
time = timer();
}
if(err.isValid() && err.isInjectedFault() && severity == SevError) {
severity = SevWarnAlways;
}
detail("Severity", severity);
detailf("Time", "%.6f", time);
detail("Type", type);
@ -664,6 +617,15 @@ bool TraceEvent::init( Severity severity, const char* type ) {
NetworkAddress local = g_network->getLocalAddress();
detailf("Machine", "%d.%d.%d.%d:%d", (local.ip>>24)&0xff, (local.ip>>16)&0xff, (local.ip>>8)&0xff, local.ip&0xff, local.port);
}
detail("ID", id);
if(err.isValid()) {
if (err.isInjectedFault()) {
detail("ErrorIsInjectedFault", true);
}
detail("Error", err.name());
detail("ErrorDescription", err.what());
detail("ErrorCode", err.code());
}
} else {
tmpEventMetric = nullptr;
}
@ -672,24 +634,31 @@ bool TraceEvent::init( Severity severity, const char* type ) {
}
TraceEvent& TraceEvent::error(class Error const& error, bool includeCancelled) {
if (enabled) {
if (error.code() == error_code_actor_cancelled && !includeCancelled) {
// Suppress the entire message
enabled = false;
} else {
if (error.isInjectedFault()) {
detail("ErrorIsInjectedFault", true);
if (severity == SevError) severity = SevWarnAlways;
if(enabled) {
if (error.code() != error_code_actor_cancelled || includeCancelled) {
err = error;
if (initialized) {
if (error.isInjectedFault()) {
detail("ErrorIsInjectedFault", true);
if(severity == SevError) severity = SevWarnAlways;
}
detail("Error", error.name());
detail("ErrorDescription", error.what());
detail("ErrorCode", error.code());
}
} else {
if (initialized) {
TraceEvent(g_network && g_network->isSimulated() ? SevError : SevWarnAlways, std::string(TRACE_EVENT_INVALID_SUPPRESSION).append(type).c_str()).suppressFor(5);
} else {
enabled = false;
}
detail("Error", error.name());
detail("ErrorDescription", error.what());
detail("ErrorCode", error.code());
}
}
return *this;
}
TraceEvent& TraceEvent::detailImpl( std::string&& key, std::string&& value, bool writeEventMetricField) {
init();
if (enabled) {
if( value.size() > 495 ) {
value = value.substr(0, 495) + "...";
@ -713,36 +682,43 @@ TraceEvent& TraceEvent::detail( std::string key, std::string value ) {
return detailImpl(std::move(key), std::move(value));
}
TraceEvent& TraceEvent::detail( std::string key, double value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), value);
return detailfNoMetric( std::move(key), "%g", value );
}
TraceEvent& TraceEvent::detail( std::string key, int value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), (int64_t)value);
return detailfNoMetric( std::move(key), "%d", value );
}
TraceEvent& TraceEvent::detail( std::string key, unsigned value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), (int64_t)value);
return detailfNoMetric( std::move(key), "%u", value );
}
TraceEvent& TraceEvent::detail( std::string key, long int value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), (int64_t)value);
return detailfNoMetric( std::move(key), "%ld", value );
}
TraceEvent& TraceEvent::detail( std::string key, long unsigned int value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), (int64_t)value);
return detailfNoMetric( std::move(key), "%lu", value );
}
TraceEvent& TraceEvent::detail( std::string key, long long int value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), (int64_t)value);
return detailfNoMetric( std::move(key), "%lld", value );
}
TraceEvent& TraceEvent::detail( std::string key, long long unsigned int value ) {
init();
if(enabled)
tmpEventMetric->setField(key.c_str(), (int64_t)value);
return detailfNoMetric( std::move(key), "%llu", value );
@ -793,33 +769,48 @@ TraceEvent& TraceEvent::trackLatest( const char *trackingKey ){
}
TraceEvent& TraceEvent::sample( double sampleRate, bool logSampleRate ) {
if(!g_random) {
sampleRate = 1.0;
}
else {
enabled = enabled && g_random->random01() < sampleRate;
}
if(enabled) {
if(initialized) {
TraceEvent(g_network && g_network->isSimulated() ? SevError : SevWarnAlways, std::string(TRACE_EVENT_INVALID_SUPPRESSION).append(type).c_str()).suppressFor(5);
return *this;
}
if(enabled && logSampleRate) {
detail("SampleRate", sampleRate);
if(!g_random) {
sampleRate = 1.0;
}
else {
enabled = enabled && g_random->random01() < sampleRate;
}
if(enabled && logSampleRate) {
detail("SampleRate", sampleRate);
}
}
return *this;
}
TraceEvent& TraceEvent::suppressFor( double duration, bool logSuppressedEventCount ) {
if(g_network) {
if(isNetworkThread()) {
int64_t suppressedEventCount = suppressedEvents.checkAndInsertSuppression(type, duration);
enabled = enabled && suppressedEventCount >= 0;
if(enabled && logSuppressedEventCount) {
detail("SuppressedEventCount", suppressedEventCount);
if(enabled) {
if(initialized) {
TraceEvent(g_network && g_network->isSimulated() ? SevError : SevWarnAlways, std::string(TRACE_EVENT_INVALID_SUPPRESSION).append(type).c_str()).suppressFor(5);
return *this;
}
if(g_network) {
if(isNetworkThread()) {
int64_t suppressedEventCount = suppressedEvents.checkAndInsertSuppression(type, duration);
enabled = enabled && suppressedEventCount >= 0;
if(enabled && logSuppressedEventCount) {
detail("SuppressedEventCount", suppressedEventCount);
}
}
else {
TraceEvent(SevError, "SuppressionFromNonNetworkThread");
detail("__InvalidSuppression__", ""); // Choosing a detail name that is unlikely to collide with other names
}
}
else {
TraceEvent(SevError, "SuppressionFromNonNetworkThread");
detail("__InvalidSuppression__", ""); // Choosing a detail name that is unlikely to collide with other names
}
init(); //we do not want any future calls on this trace event to disable it, because we have already counted it towards our suppression budget
}
return *this;
@ -842,11 +833,12 @@ unsigned long TraceEvent::CountEventsLoggedAt(Severity sev) {
}
TraceEvent& TraceEvent::backtrace(const std::string& prefix) {
if (this->severity == SevError) return *this; // We'll backtrace this later in ~TraceEvent
if (this->severity == SevError || !enabled) return *this; // We'll backtrace this later in ~TraceEvent
return detail(prefix + "Backtrace", platform::get_backtrace());
}
TraceEvent::~TraceEvent() {
init();
try {
if (enabled) {
if (this->severity == SevError) {

View File

@ -139,15 +139,11 @@ struct TraceEvent {
TraceEvent( Severity, const char* type, UID id = UID() );
TraceEvent( struct TraceInterval&, UID id = UID() );
TraceEvent( Severity severity, struct TraceInterval& interval, UID id = UID() );
TraceEvent(const char* type, const StringRef& id); // Assumes SevInfo severity
TraceEvent(Severity, const char* type, const StringRef& id);
TraceEvent(const char* type, const Optional<Standalone<StringRef>>& id); // Assumes SevInfo severity
TraceEvent(Severity, const char* type, const Optional<Standalone<StringRef>>& id);
static bool isEnabled( const char* type, Severity = SevMax );
static void setNetworkThread();
static bool isNetworkThread();
//Must be called directly after constructing the trace event
TraceEvent& error(const class Error& e, bool includeCancelled=false);
TraceEvent& detail( std::string key, std::string value );
@ -172,6 +168,8 @@ public:
TraceEvent& backtrace(const std::string& prefix = "");
TraceEvent& trackLatest( const char* trackingKey );
TraceEvent& sample( double sampleRate, bool logSampleRate=true );
//Cannot call other functions which could disable the trace event afterwords
TraceEvent& suppressFor( double duration, bool logSuppressedEventCount=true );
TraceEvent& GetLastError();
@ -184,19 +182,20 @@ public:
DynamicEventMetric *tmpEventMetric; // This just just a place to store fields
private:
bool initialized;
bool enabled;
std::string trackingKey;
TraceEventFields fields;
int length;
Severity severity;
const char *type;
UID id;
Error err;
static unsigned long eventCounts[5];
static thread_local bool networkThread;
bool init( Severity, const char* type );
bool init( Severity, struct TraceInterval& );
bool init();
bool init( struct TraceInterval& );
};
struct ITraceLogWriter {

View File

@ -41,7 +41,7 @@ Future<T> traceAfter(Future<T> what, const char* type, const char* key, X value,
TraceEvent(type).detail(key, value);
return val;
} catch( Error &e ) {
if(traceErrors) TraceEvent(type).detail(key, value).error(e,true);
if(traceErrors) TraceEvent(type).error(e,true).detail(key, value);
throw;
}
}

View File

@ -204,7 +204,7 @@ struct _IncludeVersion {
ar >> v;
if (v < minValidProtocolVersion) {
auto err = incompatible_protocol_version();
TraceEvent(SevError, "InvalidSerializationVersion").detailf("Version", "%llx", v).error(err);
TraceEvent(SevError, "InvalidSerializationVersion").error(err).detailf("Version", "%llx", v);
throw err;
}
if (v > currentProtocolVersion) {
@ -212,7 +212,7 @@ struct _IncludeVersion {
// particular data structures (e.g. to support mismatches between client and server versions when the client
// must deserialize zookeeper and database structures)
auto err = incompatible_protocol_version();
TraceEvent(SevError, "FutureProtocolVersion").detailf("Version", "%llx", v).error(err);
TraceEvent(SevError, "FutureProtocolVersion").error(err).detailf("Version", "%llx", v);
throw err;
}
ar.setProtocolVersion(v);