From bf79acd1dedcf4739cbc93b7cf10084c7fc728c3 Mon Sep 17 00:00:00 2001 From: Igor Gaponenko Date: Fri, 5 Apr 2024 00:18:15 +0000 Subject: [PATCH 01/15] Extended transient API of QMeta to read workers-to-chunks map from database Czar and workers can send http messages to each other. --- deploy/compose/docker-compose.yml | 1 + python/lsst/qserv/admin/itest.py | 1 + python/lsst/qserv/admin/qservCli/launch.py | 2 + src/ccontrol/MergingHandler.cc | 234 +++++++++- src/ccontrol/MergingHandler.h | 13 + src/ccontrol/UserQuery.h | 6 +- src/ccontrol/UserQueryAsyncResult.cc | 12 +- src/ccontrol/UserQueryAsyncResult.h | 6 +- src/ccontrol/UserQueryFactory.cc | 6 +- src/ccontrol/UserQueryInvalid.h | 8 +- src/ccontrol/UserQueryProcessList.cc | 17 +- src/ccontrol/UserQueryProcessList.h | 4 +- src/ccontrol/UserQueryResultDelete.h | 9 +- src/ccontrol/UserQuerySelect.cc | 214 ++++++++- src/ccontrol/UserQuerySelect.h | 31 +- src/ccontrol/UserQuerySelectCountStar.cc | 5 +- src/ccontrol/UserQuerySelectCountStar.h | 11 +- src/ccontrol/UserQuerySet.cc | 4 +- src/ccontrol/UserQuerySet.h | 4 +- src/czar/CMakeLists.txt | 32 ++ src/czar/Czar.cc | 113 ++++- src/czar/Czar.h | 38 ++ src/czar/CzarChunkMap.cc | 418 +++++++++++++++++ src/czar/CzarChunkMap.h | 324 ++++++++++++++ src/czar/CzarRegistry.cc | 198 ++++++++ src/czar/CzarRegistry.h | 137 ++++++ src/czar/HttpCzarWorkerModule.cc | 168 +++++++ src/czar/HttpCzarWorkerModule.h | 82 ++++ src/czar/HttpSvc.cc | 11 + src/czar/MessageTable.cc | 4 +- src/czar/testCzar.cc | 205 +++++++++ src/global/clock_defs.h | 1 + src/global/intTypes.h | 8 +- src/http/Module.h | 10 + src/http/RequestBodyJSON.h | 18 + src/proto/ScanTableInfo.h | 6 + src/proto/worker.proto | 1 + src/qdisp/CMakeLists.txt | 4 +- src/qdisp/Executive.cc | 199 ++++++-- src/qdisp/Executive.h | 132 +++++- src/qdisp/JobBase.cc | 54 +++ src/qdisp/JobBase.h | 78 ++++ src/qdisp/JobDescription.cc | 39 +- src/qdisp/JobDescription.h | 26 +- src/qdisp/JobQuery.cc | 83 +++- src/qdisp/JobQuery.h | 99 ++-- src/qdisp/ResponseHandler.h | 17 +- src/qdisp/UberJob.cc | 473 ++++++++++++++++++++ src/qdisp/UberJob.h | 173 +++++++ src/qdisp/testQDisp.cc | 28 +- src/qhttp/Server.cc | 24 + src/qmeta/CMakeLists.txt | 6 +- src/qmeta/Exceptions.h | 9 + src/{qdisp => qmeta}/JobStatus.cc | 35 +- src/{qdisp => qmeta}/JobStatus.h | 34 +- src/{qdisp => qmeta}/MessageStore.cc | 15 +- src/{qdisp => qmeta}/MessageStore.h | 18 +- src/qmeta/QMeta.h | 65 ++- src/qmeta/QMetaMysql.cc | 102 ++++- src/qmeta/QMetaMysql.h | 24 +- src/qmeta/testQMeta.cc | 7 + src/qproc/TaskMsgFactory.cc | 158 +++++++ src/qproc/TaskMsgFactory.h | 20 + src/replica/apps/MasterControllerHttpApp.cc | 6 +- src/replica/apps/MasterControllerHttpApp.h | 1 + src/replica/contr/ReplicationTask.cc | 92 +++- src/replica/contr/ReplicationTask.h | 9 +- src/replica/contr/Task.h | 6 + src/rproc/InfileMerger.cc | 113 ++++- src/rproc/InfileMerger.h | 7 + src/sql/SqlResults.cc | 18 + src/sql/SqlResults.h | 9 + src/util/TimeUtils.cc | 9 + src/util/TimeUtils.h | 6 + src/wbase/CMakeLists.txt | 1 + src/wbase/FileChannelShared.cc | 167 +++++-- src/wbase/FileChannelShared.h | 27 +- src/wbase/Task.cc | 200 ++++++++- src/wbase/Task.h | 42 +- src/wbase/UberJobData.cc | 180 ++++++++ src/wbase/UberJobData.h | 121 +++++ src/wbase/UserQueryInfo.cc | 7 + src/wbase/UserQueryInfo.h | 9 + src/wsched/ChunkTasksQueue.cc | 17 +- src/wsched/ScanScheduler.cc | 26 +- src/xrdsvc/CMakeLists.txt | 1 + src/xrdsvc/HttpReplicaMgtModule.cc | 6 +- src/xrdsvc/HttpSvc.cc | 7 + src/xrdsvc/HttpWorkerCzarModule.cc | 213 +++++++++ src/xrdsvc/HttpWorkerCzarModule.h | 88 ++++ 90 files changed, 5355 insertions(+), 317 deletions(-) create mode 100644 src/czar/CzarChunkMap.cc create mode 100644 src/czar/CzarChunkMap.h create mode 100644 src/czar/CzarRegistry.cc create mode 100644 src/czar/CzarRegistry.h create mode 100644 src/czar/HttpCzarWorkerModule.cc create mode 100644 src/czar/HttpCzarWorkerModule.h create mode 100644 src/czar/testCzar.cc create mode 100644 src/qdisp/JobBase.cc create mode 100644 src/qdisp/JobBase.h create mode 100644 src/qdisp/UberJob.cc create mode 100644 src/qdisp/UberJob.h rename src/{qdisp => qmeta}/JobStatus.cc (74%) rename src/{qdisp => qmeta}/JobStatus.h (79%) rename src/{qdisp => qmeta}/MessageStore.cc (87%) rename src/{qdisp => qmeta}/MessageStore.h (90%) create mode 100644 src/wbase/UberJobData.cc create mode 100644 src/wbase/UberJobData.h create mode 100644 src/xrdsvc/HttpWorkerCzarModule.cc create mode 100644 src/xrdsvc/HttpWorkerCzarModule.h diff --git a/deploy/compose/docker-compose.yml b/deploy/compose/docker-compose.yml index bc792c23c7..d1ea8ec323 100644 --- a/deploy/compose/docker-compose.yml +++ b/deploy/compose/docker-compose.yml @@ -457,6 +457,7 @@ services: --registry-host=repl-registry --controller-auto-register-workers=1 --qserv-sync-force + --qserv-chunk-map-update --debug expose: - "25081" diff --git a/python/lsst/qserv/admin/itest.py b/python/lsst/qserv/admin/itest.py index cb7d9c5b04..900783087b 100644 --- a/python/lsst/qserv/admin/itest.py +++ b/python/lsst/qserv/admin/itest.py @@ -1084,6 +1084,7 @@ def compare_query_results(run_cases: list[str], outputs_dir: str) -> list[ITestC if not os.path.exists(os.path.join(outputs_dir, case)): _log.warn("There are no query results to compare for %s", case) continue + comparisons = ( (query_mode_mysql, query_mode_qserv_attached), (query_mode_mysql, query_mode_qserv_detached), diff --git a/python/lsst/qserv/admin/qservCli/launch.py b/python/lsst/qserv/admin/qservCli/launch.py index 6ac9008e9f..cf26032195 100644 --- a/python/lsst/qserv/admin/qservCli/launch.py +++ b/python/lsst/qserv/admin/qservCli/launch.py @@ -254,7 +254,9 @@ def cmake( build_image, "cmake", "..", + "-DCMAKE_BUILD_TYPE=Debug" ] + # "-DCMAKE_BUILD_TYPE=Debug" if dry: print(" ".join(args)) return diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index b67f519704..3e79478139 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -53,6 +53,7 @@ #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" #include "qdisp/QueryRequest.h" +#include "qdisp/UberJob.h" #include "rproc/InfileMerger.h" #include "util/Bug.h" #include "util/common.h" @@ -158,6 +159,166 @@ string readHttpFileAndMerge(string const& httpUrl, size_t fileSize, return string(); } +std::tuple readHttpFileAndMergeHttp( + lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, + function const& messageIsReady, + shared_ptr const& httpConnPool) { + string const context = "MergingHandler::" + string(__func__) + " " + " qid=" + uberJob->getIdStr() + " "; + + LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); + + // Track the file while the control flow is staying within the function. + ResultFileTracker const resultFileTracker; + + // The data transmit rate tracker is set up before reading each data message. + unique_ptr> transmitRateTracker; + + // A location of the next byte to be read from the input file. The variable + // is used for error reporting. + uint64_t offset = 0; + + // Temporary buffer for messages read from the file. The buffer gets automatically + // resized to fit the largest message. + unique_ptr msgBuf; + size_t msgBufSize = 0; + size_t msgBufNext = 0; // An index of the next character in the buffer. + + // Fixed-size buffer to store the message size. + string msgSizeBuf(sizeof(uint32_t), '\0'); + size_t msgSizeBufNext = 0; // An index of the next character in the buffer. + + // The size of the next/current message. The variable is set after succesfully parsing + // the message length header and is reset back to 0 after parsing the message body. + // The value is stays 0 while reading the frame header. + uint32_t msgSizeBytes = 0; + bool success = true; + bool mergeSuccess = true; + int headerCount = 0; + uint64_t totalBytesRead = 0; + try { + string const noClientData; + vector const noClientHeaders; + http::ClientConfig clientConfig; + clientConfig.httpVersion = CURL_HTTP_VERSION_1_1; // same as in qhttp + clientConfig.bufferSize = CURL_MAX_READ_SIZE; // 10 MB in the current version of libcurl + clientConfig.tcpKeepAlive = true; + clientConfig.tcpKeepIdle = 5; // the default is 60 sec + clientConfig.tcpKeepIntvl = 5; // the default is 60 sec + http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, + httpConnPool); + reader.read([&](char const* inBuf, size_t inBufSize) { + // A value of the flag is set by the message processor when it's time to finish + // or abort reading the file. + bool last = false; + char const* next = inBuf; + char const* const end = inBuf + inBufSize; + while ((next < end) && !last) { + LOGS(_log, LOG_LVL_WARN, + context << "TODO:UJ next=" << (uint64_t)next << " end=" << (uint64_t)end + << " last=" << last); + if (msgSizeBytes == 0) { + // Continue or finish reading the frame header. + size_t const bytes2read = + std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); + std::memcpy(msgSizeBuf.data() + msgSizeBufNext, next, bytes2read); + next += bytes2read; + offset += bytes2read; + msgSizeBufNext += bytes2read; + if (msgSizeBufNext == sizeof(uint32_t)) { + ++headerCount; + // Done reading the frame header. + msgSizeBufNext = 0; + // Parse and evaluate the message length. + msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); + if (msgSizeBytes == 0) { + throw runtime_error("message size is 0 at offset " + + to_string(offset - sizeof(uint32_t)) + ", file: " + httpUrl); + } + if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { + throw runtime_error("message size " + to_string(msgSizeBytes) + " at offset " + + to_string(offset - sizeof(uint32_t)) + + " exceeds the hard limit of " + + to_string(ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) + + ", file: " + httpUrl); + } + // Extend the message buffer (if needed). Note that buffer never gets + // truncated to avoid excessive memory deallocations/allocations. + if (msgBufSize < msgSizeBytes) { + msgBufSize = msgSizeBytes; + msgBuf.reset(new char[msgBufSize]); + } + // Starts the tracker to measure the performance of the network I/O. + transmitRateTracker = + make_unique>(reportFileRecvRate); + } + } else { + // Continue or finish reading the message body. + size_t const bytes2read = + std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); + std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); + next += bytes2read; + offset += bytes2read; + msgBufNext += bytes2read; + if (msgBufNext == msgSizeBytes) { + // Done reading message body. + msgBufNext = 0; + + // Destroying the tracker will result in stopping the tracker's timer and + // reporting the file read rate before proceeding to the merge. + if (transmitRateTracker != nullptr) { + transmitRateTracker->addToValue(msgSizeBytes); + transmitRateTracker->setSuccess(); + transmitRateTracker.reset(); + } + + // Parse and evaluate the message. + mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); + totalBytesRead += msgSizeBytes; + if (!mergeSuccess) { + success = false; + throw runtime_error("message processing failed at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + // Reset the variable to prepare for reading the next header & message (if any). + msgSizeBytes = 0; + } else { + LOGS(_log, LOG_LVL_WARN, + context << " headerCount=" << headerCount + << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); + } + } + } + }); + LOGS(_log, LOG_LVL_DEBUG, + context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes + << " totalBytesRead=" << totalBytesRead); + if (msgSizeBufNext != 0) { + throw runtime_error("short read of the message header at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + if (msgBufNext != 0) { + throw runtime_error("short read of the message body at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); + success = false; + } + + // Remove the file from the worker if it still exists. Report and ignore errors. + // The files will be garbage-collected by workers. + try { + http::Client remover(http::Method::DELETE, httpUrl); + remover.read([](char const* inBuf, size_t inBufSize) {}); + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, context << "failed to remove " << httpUrl << ", ex: " << ex.what()); + } + // If the merge failed, that indicates something went wrong in the local database table, + // is likely this user query is doomed and should be cancelled. + LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeSuccess); + return {success, mergeSuccess}; +} + } // namespace namespace lsst::qserv::ccontrol { @@ -167,7 +328,7 @@ MergingHandler::MergingHandler(std::shared_ptr merger, std: _initState(); } -MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_DEBUG, __func__); } +MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << _tableName); } bool MergingHandler::flush(proto::ResponseSummary const& resp) { _wName = resp.wname(); @@ -179,6 +340,8 @@ bool MergingHandler::flush(proto::ResponseSummary const& resp) { LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobQuery was NULL"); return false; } + auto const jobQuery = std::dynamic_pointer_cast(jobBase); + LOGS(_log, LOG_LVL_TRACE, "MergingHandler::" << __func__ << " jobid=" << resp.jobid() << " transmitsize=" << resp.transmitsize() << " rowcount=" << resp.rowcount() << " rowSize=" @@ -314,10 +477,79 @@ bool MergingHandler::_merge(proto::ResponseSummary const& resp, shared_ptr const& uberJob, + proto::ResponseData const& responseData) { + if (_flushed) { + throw util::Bug(ERR_LOC, "already flushed"); + } + bool const success = _infileMerger->mergeHttp(uberJob, responseData); + if (!success) { + LOGS(_log, LOG_LVL_WARN, __func__ << " failed"); + util::Error const& err = _infileMerger->getError(); + _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg()); + } + return success; +} + void MergingHandler::_setError(int code, std::string const& msg) { LOGS(_log, LOG_LVL_DEBUG, "_setErr: code: " << code << ", message: " << msg); std::lock_guard lock(_errorMutex); _error = Error(code, msg); } +tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) { + bool success = false; + bool shouldCancel = false; + + // This is needed to ensure the job query would be staying alive for the duration + // of the operation to prevent inconsistency within the application. + auto const jobBase = getJobBase().lock(); + if (jobBase == nullptr) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobBase was NULL"); + return {success, shouldCancel}; // both should still be false + } + auto const uberJob = std::dynamic_pointer_cast(jobBase); + + LOGS(_log, LOG_LVL_TRACE, + "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); + + // Dispatch result processing to the corresponidng method which depends on + // the result delivery protocol configured at the worker. + // Notify the file reader when all rows have been read by setting 'last = true'. + auto const dataMergerHttp = [&](char const* buf, uint32_t bufSize, bool& last) { + LOGS(_log, LOG_LVL_TRACE, "dataMergerHttp"); + last = true; + proto::ResponseData responseData; + if (responseData.ParseFromArray(buf, bufSize) && responseData.IsInitialized()) { + bool const mergeSuccess = _mergeHttp(uberJob, responseData); + if (mergeSuccess) { + resultRows += responseData.row_size(); + last = resultRows >= expectedRows; + } + return mergeSuccess; + } + throw runtime_error("MergingHandler::flush ** message deserialization failed **"); + }; + + tie(success, shouldCancel) = + ::readHttpFileAndMergeHttp(uberJob, fileUrl, dataMergerHttp, MergingHandler::_getHttpConnPool()); + + if (!success || shouldCancel) { + LOGS(_log, LOG_LVL_WARN, __func__ << " success=" << success << " shouldCancel=" << shouldCancel); + } + + if (success) { + _infileMerger->mergeCompleteFor(uberJob->getJobId()); + } + return {success, shouldCancel}; +} + +void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int status) { + if (!_errorSet.exchange(true)) { + _error = util::Error(errorCode, errorMsg, util::ErrorCode::MYSQLEXEC); + _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); + } +} + } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index a8f37d1d42..8c0e8dc239 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -40,6 +40,7 @@ class ResponseSummary; namespace lsst::qserv::qdisp { class JobQuery; +class UberJob; } // namespace lsst::qserv::qdisp namespace lsst::qserv::rproc { @@ -70,6 +71,14 @@ class MergingHandler : public qdisp::ResponseHandler { /// @return true if successful (no error) bool flush(proto::ResponseSummary const& resp) override; + /// @see ResponseHandler::flushHttp + /// @see MerginHandler::_mergeHttp + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) override; + + /// @see ResponseHandler::flushHttpError + void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; + /// Signal an unrecoverable error condition. No further calls are expected. void errorFlush(std::string const& msg, int code) override; @@ -93,6 +102,9 @@ class MergingHandler : public qdisp::ResponseHandler { bool _merge(proto::ResponseSummary const& resp, std::shared_ptr const& jobQuery); + /// Call InfileMerger to do the work of merging this data to the result. + bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + /// Set error code and string. void _setError(int code, std::string const& msg); @@ -106,6 +118,7 @@ class MergingHandler : public qdisp::ResponseHandler { std::shared_ptr _infileMerger; ///< Merging delegate std::string _tableName; ///< Target table name Error _error; ///< Error description + std::atomic _errorSet{false}; ///< Set to true when an error is set. mutable std::mutex _errorMutex; ///< Protect readers from partial updates bool _flushed{false}; ///< flushed to InfileMerger? std::string _wName{"~"}; ///< worker name diff --git a/src/ccontrol/UserQuery.h b/src/ccontrol/UserQuery.h index 1989916884..a63efa2bd8 100644 --- a/src/ccontrol/UserQuery.h +++ b/src/ccontrol/UserQuery.h @@ -42,9 +42,9 @@ #include "qmeta/types.h" // Forward decl -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { class MessageStore; -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qmeta namespace lsst::qserv::ccontrol { @@ -74,7 +74,7 @@ class UserQuery { virtual void discard() = 0; // Delegate objects - virtual std::shared_ptr getMessageStore() = 0; + virtual std::shared_ptr getMessageStore() = 0; /// This method should disappear when we start supporting results /// in locations other than MySQL tables. We'll switch to getResultLocation() diff --git a/src/ccontrol/UserQueryAsyncResult.cc b/src/ccontrol/UserQueryAsyncResult.cc index 0b97ba16ec..6cde104a2f 100644 --- a/src/ccontrol/UserQueryAsyncResult.cc +++ b/src/ccontrol/UserQueryAsyncResult.cc @@ -33,9 +33,9 @@ // Qserv headers #include "cconfig/CzarConfig.h" #include "qmeta/Exceptions.h" +#include "qmeta/JobStatus.h" #include "qmeta/QMeta.h" -#include "qdisp/JobStatus.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" #include "sql/SqlResults.h" @@ -53,7 +53,7 @@ UserQueryAsyncResult::UserQueryAsyncResult(QueryId queryId, qmeta::CzarId czarId _queryId(queryId), _czarId(czarId), _qMeta(qMeta), - _messageStore(std::make_shared()) { + _messageStore(std::make_shared()) { LOGS(_log, LOG_LVL_DEBUG, "UserQueryAsyncResult: QID=" << queryId); // get query info from QMeta @@ -151,8 +151,8 @@ void UserQueryAsyncResult::submit() { std::string sevStr = row[3].first; int64_t timestampMilli = boost::lexical_cast(row[4].first); MessageSeverity sev = sevStr == "INFO" ? MSG_INFO : MSG_ERROR; - qdisp::JobStatus::Clock::duration duration = std::chrono::milliseconds(timestampMilli); - qdisp::JobStatus::TimeType timestamp(duration); + qmeta::JobStatus::Clock::duration duration = std::chrono::milliseconds(timestampMilli); + qmeta::JobStatus::TimeType timestamp(duration); _messageStore->addMessage(chunkId, "DUPLICATE", code, message, sev, timestamp); } catch (std::exception const& exc) { LOGS(_log, LOG_LVL_ERROR, "Error reading message table data: " << exc.what()); @@ -186,7 +186,7 @@ void UserQueryAsyncResult::kill() {} void UserQueryAsyncResult::discard() {} -std::shared_ptr UserQueryAsyncResult::getMessageStore() { return _messageStore; } +std::shared_ptr UserQueryAsyncResult::getMessageStore() { return _messageStore; } std::string UserQueryAsyncResult::getResultTableName() const { if (_qInfo.resultLocation().compare(0, 6, "table:") == 0) { diff --git a/src/ccontrol/UserQueryAsyncResult.h b/src/ccontrol/UserQueryAsyncResult.h index 683ef5ba57..b481a8066b 100644 --- a/src/ccontrol/UserQueryAsyncResult.h +++ b/src/ccontrol/UserQueryAsyncResult.h @@ -31,7 +31,7 @@ #include "qmeta/QInfo.h" #include "qmeta/types.h" -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { class MessageStore; } @@ -84,7 +84,7 @@ class UserQueryAsyncResult : public UserQuery { void discard() override; // Delegate objects - std::shared_ptr getMessageStore() override; + std::shared_ptr getMessageStore() override; /// This method should disappear when we start supporting results /// in locations other than MySQL tables. We'll switch to getResultLocation() @@ -106,7 +106,7 @@ class UserQueryAsyncResult : public UserQuery { qmeta::CzarId _czarId; std::shared_ptr _qMeta; qmeta::QInfo _qInfo; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; QueryState _qState = UNKNOWN; }; diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 28479d122b..a5fcf1fc90 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -51,10 +51,11 @@ #include "ccontrol/UserQueryType.h" #include "css/CssAccess.h" #include "css/KvInterfaceImplMem.h" +#include "czar/Czar.h" #include "mysql/MySqlConfig.h" #include "parser/ParseException.h" #include "qdisp/Executive.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "qmeta/QMetaMysql.h" #include "qmeta/QMetaSelect.h" #include "qmeta/QProgress.h" @@ -351,7 +352,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st sessionValid = false; } - auto messageStore = std::make_shared(); + auto messageStore = std::make_shared(); std::shared_ptr executive; std::shared_ptr infileMergerConfig; if (sessionValid) { @@ -375,6 +376,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st uq->qMetaRegister(resultLocation, msgTableName); uq->setupMerger(); uq->saveResultQuery(); + executive->setUserQuerySelect(uq); } return uq; } else if (UserQueryType::isSelectResult(query, userJobId)) { diff --git a/src/ccontrol/UserQueryInvalid.h b/src/ccontrol/UserQueryInvalid.h index 3296bf47f7..1ab69bb943 100644 --- a/src/ccontrol/UserQueryInvalid.h +++ b/src/ccontrol/UserQueryInvalid.h @@ -32,7 +32,7 @@ // Qserv headers #include "ccontrol/UserQuery.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "qmeta/types.h" // Forward decl @@ -44,7 +44,7 @@ namespace lsst::qserv::ccontrol { class UserQueryInvalid : public UserQuery { public: UserQueryInvalid(std::string const& message) - : _message(message), _messageStore(std::make_shared()) {} + : _message(message), _messageStore(std::make_shared()) {} UserQueryInvalid(UserQueryInvalid const&) = delete; UserQueryInvalid& operator=(UserQueryInvalid const&) = delete; @@ -69,11 +69,11 @@ class UserQueryInvalid : public UserQuery { virtual void discard() override {} // Delegate objects - virtual std::shared_ptr getMessageStore() override { return _messageStore; } + virtual std::shared_ptr getMessageStore() override { return _messageStore; } private: std::string const _message; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQueryProcessList.cc b/src/ccontrol/UserQueryProcessList.cc index d83919dc30..88c37c5ea0 100644 --- a/src/ccontrol/UserQueryProcessList.cc +++ b/src/ccontrol/UserQueryProcessList.cc @@ -35,8 +35,7 @@ // Qserv headers #include "css/CssAccess.h" #include "css/CssError.h" -#include "cconfig/CzarConfig.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "qmeta/Exceptions.h" #include "qmeta/QMetaSelect.h" #include "query/FromList.h" @@ -66,9 +65,10 @@ UserQueryProcessList::UserQueryProcessList(std::shared_ptr co std::shared_ptr const& qMetaSelect, qmeta::CzarId czarId, std::string const& userQueryId, std::string const& resultDb) - : _qMetaSelect(qMetaSelect), - _czarId(czarId), - _messageStore(std::make_shared()), + : _resultDbConn(resultDbConn), + _qMetaSelect(qMetaSelect), + _qMetaCzarId(qMetaCzarId), + _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), _resultDb(resultDb) { // The SQL statement should be mostly OK alredy but we need to change @@ -93,9 +93,10 @@ UserQueryProcessList::UserQueryProcessList(std::shared_ptr co UserQueryProcessList::UserQueryProcessList(bool full, std::shared_ptr const& qMetaSelect, qmeta::CzarId czarId, std::string const& userQueryId, std::string const& resultDb) - : _qMetaSelect(qMetaSelect), - _czarId(czarId), - _messageStore(std::make_shared()), + : _resultDbConn(resultDbConn), + _qMetaSelect(qMetaSelect), + _qMetaCzarId(qMetaCzarId), + _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), _resultDb(resultDb) { _query = "SELECT `qi`.`queryId` `ID`,`qi`.`qType` `TYPE`,`qc`.`czar` `CZAR`,`qc`.`czarId` `CZAR_ID`," diff --git a/src/ccontrol/UserQueryProcessList.h b/src/ccontrol/UserQueryProcessList.h index 67ac9cf058..e502605d17 100644 --- a/src/ccontrol/UserQueryProcessList.h +++ b/src/ccontrol/UserQueryProcessList.h @@ -95,7 +95,7 @@ class UserQueryProcessList : public UserQuery { void discard() override; // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } /// @return Name of the result table for this query, can be empty std::string getResultTableName() const override { return _resultTableName; } @@ -113,7 +113,7 @@ class UserQueryProcessList : public UserQuery { std::shared_ptr _qMetaSelect; qmeta::CzarId const _czarId; ///< Czar ID in QMeta database QueryState _qState = UNKNOWN; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; std::string _resultTableName; std::string _query; ///< query to execute on QMeta database std::string _orderBy; diff --git a/src/ccontrol/UserQueryResultDelete.h b/src/ccontrol/UserQueryResultDelete.h index 759d391a07..66b51a6131 100644 --- a/src/ccontrol/UserQueryResultDelete.h +++ b/src/ccontrol/UserQueryResultDelete.h @@ -38,9 +38,9 @@ #include "ccontrol/QueryState.h" #include "global/intTypes.h" -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { class MessageStore; -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qmeta namespace lsst::qserv::ccontrol { @@ -74,12 +74,13 @@ class UserQueryResultDelete : public UserQuery { void discard() override {} // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } private: std::string const _value; std::shared_ptr const _queryResources; - std::shared_ptr _messageStore; + std::string _resultTableName; + std::shared_ptr _messageStore; QueryState _qState{UNKNOWN}; }; diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 4a92454812..8a51f8e597 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -80,12 +80,17 @@ #include "ccontrol/MergingHandler.h" #include "ccontrol/TmpTableName.h" #include "ccontrol/UserQueryError.h" +#include "czar/Czar.h" +#include "czar/CzarChunkMap.h" +#include "czar/CzarRegistry.h" #include "global/constants.h" #include "global/LogContext.h" #include "proto/worker.pb.h" #include "proto/ProtoImporter.h" #include "qdisp/Executive.h" -#include "qdisp/MessageStore.h" +#include "qdisp/JobQuery.h" +#include "qmeta/MessageStore.h" +#include "qmeta/QMeta.h" #include "qmeta/Exceptions.h" #include "qmeta/QMeta.h" #include "qmeta/QProgress.h" @@ -103,14 +108,18 @@ #include "query/ValueFactor.h" #include "rproc/InfileMerger.h" #include "sql/Schema.h" +#include "util/Bug.h" #include "util/IterableFormatter.h" #include "util/ThreadPriority.h" #include "xrdreq/QueryManagementAction.h" +#include "qdisp/UberJob.h" namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.ccontrol.UserQuerySelect"); } // namespace +using namespace std; + namespace lsst::qserv { /// A class that can be used to parameterize a ProtoImporter for @@ -128,7 +137,7 @@ namespace ccontrol { /// Constructor UserQuerySelect::UserQuerySelect(std::shared_ptr const& qs, - std::shared_ptr const& messageStore, + std::shared_ptr const& messageStore, std::shared_ptr const& executive, std::shared_ptr const& dbModels, std::shared_ptr const& infileMergerConfig, @@ -243,24 +252,18 @@ void UserQuerySelect::submit() { assert(_infileMerger); auto taskMsgFactory = std::make_shared(); - TmpTableName ttn(_queryId, _qSession->getOriginal()); - int sequence = 0; - auto queryTemplates = _qSession->makeQueryTemplates(); + _ttn = std::make_shared(_qMetaQueryId, _qSession->getOriginal()); + std::vector chunks; + std::mutex chunksMtx; + JobId sequence = 0; + auto queryTemplates = _qSession->makeQueryTemplates(); LOGS(_log, LOG_LVL_DEBUG, "first query template:" << (queryTemplates.size() > 0 ? queryTemplates[0].sqlFragment() : "none produced.")); // Writing query for each chunk, stop if query is cancelled. - // attempt to change priority, requires root - bool increaseThreadPriority = false; // TODO: add to configuration - util::ThreadPriority threadPriority(pthread_self()); - if (increaseThreadPriority) { - threadPriority.storeOriginalValues(); - threadPriority.setPriorityPolicy(10); - } - - // Add QProgress table entry + // Add QStatsTmp table entry try { _queryProgress->insert(_queryId, _qSession->getChunksSize()); } catch (qmeta::SqlError const& e) { @@ -269,6 +272,9 @@ void UserQuerySelect::submit() { _executive->setScanInteractive(_qSession->getScanInteractive()); + string dbName(""); + bool dbNameSet = false; + for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); ++i) { auto& chunkSpec = *i; @@ -292,20 +298,188 @@ void UserQuerySelect::submit() { _executive->add(jobDesc); }; - auto cmd = std::make_shared(funcBuildJob); - _executive->queueJobStart(cmd); + if (!uberJobsEnabled) { + // references in captures cause races + auto funcBuildJob = [this, job{move(job)}](util::CmdData*) { + QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); + _executive->runJobQuery(job); + }; + auto cmd = std::make_shared(funcBuildJob); + _executive->queueJobStart(cmd); + } ++sequence; } - // attempt to restore original thread priority, requires root - if (increaseThreadPriority) { - threadPriority.restoreOriginalValues(); + if (dbNameSet) { + _queryDbName = dbName; + } + + /// At this point the executive has a map of all jobs with the chunkIds as the key. + if (uberJobsEnabled) { + // TODO:UJ _maxCHunksPerUberJob maybe put in config??? or set on command line?? + // Different queries may benefit from different values + // Such as LIMIT=1 may work best with this at 1, where + // 100 would be better for others. + _maxChunksPerUberJob = 2; + // This is needed to prevent Czar::_monitor from starting things before they are ready. + _executive->setReadyToExecute(); + buildAndSendUberJobs(); } LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); + // TODO:UJ Waiting for all jobs to start may not be needed anymore? _executive->waitForAllJobsToStart(); } +void UserQuerySelect::buildAndSendUberJobs() { + string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start"); + + // Ensure `_monitor()` doesn't do anything until everything is ready. + if (!_executive->isReadyToExecute()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive isn't ready to generate UberJobs."); + return; + } + + // Only one thread should be generating UberJobs for this user query at any given time. + lock_guard fcLock(_buildUberJobMtx); + bool const clearFlag = false; + _executive->setFlagFailedUberJob(clearFlag); + LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << _executive->getTotalJobs()); + + vector uberJobs; + + auto czarPtr = czar::Czar::getCzar(); + auto czFamilyMap = czarPtr->getCzarFamilyMap(); + auto czChunkMap = czFamilyMap->getChunkMap(_queryDbName); + auto czRegistry = czarPtr->getCzarRegistry(); + + if (czChunkMap == nullptr) { + LOGS(_log, LOG_LVL_ERROR, funcN << " no map found for queryDbName=" << _queryDbName); + // Make an empty chunk map so all jobs are flagged as needing to be reassigned. + // There's a chance that a family will be replicated by the registry. + czChunkMap = czar::CzarChunkMap::create(); + } + + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); + // Make a map of all jobs in the executive. + // TODO:UJ Maybe a check should be made that all datbases are in the same family? + + qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); + + // keep cycling through workers until no more chunks to place. + // - create a map of UberJobs key=, val=> + // - for chunkId in `unassignedChunksInQuery` + // - use `chunkMapPtr` to find the shared scan workerId for chunkId + // - if not existing in the map, make a new uberjob + // - if existing uberjob at max jobs, create a new uberjob + // - once all chunks in the query have been put in uberjobs, find contact info + // for each worker + // - add worker to each uberjob. + // - For failures - If a worker cannot be contacted, that's an uberjob failure. + // - uberjob failures (due to communications problems) will result in the uberjob + // being broken up into multiple UberJobs going to different workers. + // - The best way to do this is probably to just kill the UberJob and mark all + // Jobs that were in that UberJob as needing re-assignment, and re-running + // the code here. The trick is going to be figuring out which workers are alive. + // Maybe force a fresh lookup from the replicator Registry when an UberJob fails. + map> workerJobMap; + vector missingChunks; + + // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in + // numerical order. The workers run shared scans in numerical order of chunk id numbers. + // This keeps the number of partially complete UberJobs running on a worker to a minimum, + // and should minimize the time for the first UberJob on the worker to complete. + for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { + auto iter = chunkMapPtr->find(chunkId); + if (iter == chunkMapPtr->end()) { + missingChunks.push_back(chunkId); + bool const increaseAttemptCount = true; + jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); + // Assign as many jobs as possible. Any chunks not found will be attempted later. + continue; + } + czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; + auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + // TODO:UJ maybe if (targetWorker == nullptr || this worker already tried for this chunk) { + if (targetWorker == nullptr) { + LOGS(_log, LOG_LVL_ERROR, funcN << " No primary scan worker for chunk=" << chunkData->dump()); + // Try to assign a different worker to this job + auto workerHasThisChunkMap = chunkData->getWorkerHasThisMapCopy(); + bool found = false; + for (auto wIter = workerHasThisChunkMap.begin(); wIter != workerHasThisChunkMap.end() && !found; + ++wIter) { + auto maybeTarg = wIter->second.lock(); + if (maybeTarg != nullptr) { + targetWorker = maybeTarg; + found = true; + LOGS(_log, LOG_LVL_WARN, + funcN << " Alternate worker found for chunk=" << chunkData->dump()); + } + } + if (!found) { + // If too many workers are down, there will be a chunk that cannot be found. + // Just continuing should leave jobs `unassigned` with their attempt count + // increased. Either the chunk will be found and jobs assigned, or the jobs' + // attempt count will reach max and the query will be cancelled + // TODO:UJ Needs testing/verification + LOGS(_log, LOG_LVL_ERROR, + funcN << " No primary or alternate worker found for chunk=" << chunkData->dump()); + continue; + } + } + // Add this job to the appropriate UberJob, making the UberJob if needed. + string workerId = targetWorker->getWorkerId(); + auto& ujVect = workerJobMap[workerId]; + if (ujVect.empty() || ujVect.back()->getJobCount() >= _maxChunksPerUberJob) { + auto ujId = _uberJobIdSeq++; // keep ujId consistent + string uberResultName = _ttn->make(ujId); + auto respHandler = make_shared(_infileMerger, uberResultName); + auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), ujId, + _qMetaCzarId, targetWorker); + ujVect.push_back(uJob); + } + auto& ujVectBack = ujVect.back(); + ujVectBack->addJob(jqPtr); + LOGS(_log, LOG_LVL_DEBUG, + funcN << " ujVectBack{" << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() + << "}"); + } + + if (!missingChunks.empty()) { + string errStr = funcN + " a worker could not be found for these chunks "; + for (auto const& chk : missingChunks) { + errStr += to_string(chk) + ","; + } + errStr += " they will be retried later."; + LOGS(_log, LOG_LVL_ERROR, errStr); + // There are likely to be unassigned jobs, so set a flag to try to make + // new uber jobs for these jobs. + _executive->setFlagFailedUberJob(true); + } + + // Add worker contact info to UberJobs. + auto const wContactMap = czRegistry->getWorkerContactMap(); + LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); + for (auto const& [wIdKey, ujVect] : workerJobMap) { + auto iter = wContactMap->find(wIdKey); + if (iter == wContactMap->end()) { + // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. + // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, + // so maybe return internal error to the user? + throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + wIdKey); + } + auto const& wContactInfo = iter->second; + for (auto const& ujPtr : ujVect) { + ujPtr->setWorkerContactInfo(wContactInfo); + } + _executive->addUberJobs(ujVect); + for (auto const& ujPtr : ujVect) { + _executive->runUberJob(ujPtr); + } + } +} + /// Block until a submit()'ed query completes. /// @return the QueryState indicating success or failure QueryState UserQuerySelect::join() { @@ -392,9 +566,11 @@ void UserQuerySelect::discard() { if (_executive && _executive->getNumInflight() > 0) { throw UserQueryError(getQueryIdString() + " Executive unfinished, cannot discard"); } + _executive.reset(); _messageStore.reset(); _qSession.reset(); + try { _discardMerger(lock); } catch (UserQueryError const& e) { diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index 9887912caa..bb4910b44b 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -32,6 +32,7 @@ */ // System headers +#include #include #include #include @@ -49,11 +50,11 @@ // Forward declarations namespace lsst::qserv::qdisp { class Executive; -class MessageStore; class QdispPool; } // namespace lsst::qserv::qdisp namespace lsst::qserv::qmeta { +class MessageStore; class QMeta; class QProgress; } // namespace lsst::qserv::qmeta @@ -76,11 +77,13 @@ class InfileMergerConfig; namespace lsst::qserv::ccontrol { +class TmpTableName; + /// UserQuerySelect : implementation of the UserQuery for regular SELECT statements. class UserQuerySelect : public UserQuery { public: UserQuerySelect(std::shared_ptr const& qs, - std::shared_ptr const& messageStore, + std::shared_ptr const& messageStore, std::shared_ptr const& executive, std::shared_ptr const& dbModels, std::shared_ptr const& infileMergerConfig, @@ -119,7 +122,7 @@ class UserQuerySelect : public UserQuery { void discard() override; // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } /// @return Name of the result table for this query, can be empty std::string getResultTableName() const override { return _resultTable; } @@ -146,6 +149,11 @@ class UserQuerySelect : public UserQuery { /// save the result query in the query metadata void saveResultQuery(); + /// Use the query and jobs information in the executive to construct and run whatever + /// UberJobs are needed. This can be called multiple times by Czar::_monitor + /// to reassign failed jobs or jobs that were never assigned. + void buildAndSendUberJobs(); + private: /// @return ORDER BY part of SELECT statement that gets executed by the proxy std::string _getResultOrderBy() const; @@ -166,7 +174,7 @@ class UserQuerySelect : public UserQuery { // Delegate classes std::shared_ptr _qSession; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; std::shared_ptr _executive; std::shared_ptr _databaseModels; std::shared_ptr _infileMergerConfig; @@ -183,8 +191,21 @@ class UserQuerySelect : public UserQuery { mutable std::string _errorExtra; ///< Additional error information std::string _resultTable; ///< Result table name std::string _resultLoc; ///< Result location - std::string _resultDb; ///< Result database (todo is this the same as resultLoc??) + std::string _resultDb; ///< Result database TODO:UJ same as resultLoc??) bool _async; ///< true for async query + + /// TODO:UJ The maximum number of chunks allowed in an UberJob. At the very + /// least, this needs to be set in the configuration. However, it may also + /// be useful to change this based on the nature of each UserQuery. + int _maxChunksPerUberJob = 1; + std::atomic _uberJobIdSeq{1}; ///< Sequence number for UberJobs in this query. + std::shared_ptr _ttn; ///< Temporary table name generator. + + /// Primary database name for the query. + std::string _queryDbName; + + /// Only one thread should run buildAndSendUberJobs() for this query at a time. + std::mutex _buildUberJobMtx; }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQuerySelectCountStar.cc b/src/ccontrol/UserQuerySelectCountStar.cc index 1e19106f04..0396d34e7e 100644 --- a/src/ccontrol/UserQuerySelectCountStar.cc +++ b/src/ccontrol/UserQuerySelectCountStar.cc @@ -33,7 +33,7 @@ #include "ccontrol/UserQueryError.h" #include "ccontrol/UserQueryType.h" #include "global/LogContext.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "qmeta/QInfo.h" #include "qmeta/QMetaSelect.h" #include "query/SelectStmt.h" @@ -61,7 +61,8 @@ UserQuerySelectCountStar::UserQuerySelectCountStar(std::string query, bool async) : _qMetaSelect(qMetaSelect), _queryMetadata(queryMetadata), - _messageStore(std::make_shared()), + _messageStore(std::make_shared()), + _resultTableName(::g_nextResultTableId(userQueryId)), _userQueryId(userQueryId), _rowsTable(rowsTable), _resultDb(resultDb), diff --git a/src/ccontrol/UserQuerySelectCountStar.h b/src/ccontrol/UserQuerySelectCountStar.h index 0ad7e61266..f249bf3db8 100644 --- a/src/ccontrol/UserQuerySelectCountStar.h +++ b/src/ccontrol/UserQuerySelectCountStar.h @@ -34,12 +34,10 @@ // Forward decl namespace lsst::qserv { -namespace qdisp { -class MessageStore; -} namespace qmeta { +class MessageStore; class QMetaSelect; -} +} // namespace qmeta namespace query { class SelectStmt; } @@ -79,7 +77,7 @@ class UserQuerySelectCountStar : public UserQuery { void discard() override {} // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } /// This method should disappear when we start supporting results /// in locations other than MySQL tables. We'll switch to getResultLocation() @@ -118,9 +116,10 @@ class UserQuerySelectCountStar : public UserQuery { std::shared_ptr _qMetaSelect; std::shared_ptr const& _queryMetadata; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; std::string _resultTable; std::string _resultLoc; ///< Result location + std::string _resultTableName; std::string _userQueryId; std::string _rowsTable; std::string _resultDb; diff --git a/src/ccontrol/UserQuerySet.cc b/src/ccontrol/UserQuerySet.cc index 6007e66a6f..ce6eb55bde 100644 --- a/src/ccontrol/UserQuerySet.cc +++ b/src/ccontrol/UserQuerySet.cc @@ -22,11 +22,11 @@ #include "UserQuerySet.h" // Qserv headers -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" namespace lsst::qserv::ccontrol { UserQuerySet::UserQuerySet(std::string const& varName, std::string const& varValue) - : _varName(varName), _varValue(varValue), _messageStore(std::make_shared()) {} + : _varName(varName), _varValue(varValue), _messageStore(std::make_shared()) {} } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQuerySet.h b/src/ccontrol/UserQuerySet.h index 1d7748c1a6..f9d50a9ffb 100644 --- a/src/ccontrol/UserQuerySet.h +++ b/src/ccontrol/UserQuerySet.h @@ -68,7 +68,7 @@ class UserQuerySet : public UserQuery { void discard() override {} // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } /// This method should disappear when we start supporting results /// in locations other than MySQL tables. We'll switch to getResultLocation() @@ -91,7 +91,7 @@ class UserQuerySet : public UserQuery { std::string _varName; std::string _varValue; QueryState _qState{SUCCESS}; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; }; } // namespace lsst::qserv::ccontrol diff --git a/src/czar/CMakeLists.txt b/src/czar/CMakeLists.txt index 645b6181ca..15210908f3 100644 --- a/src/czar/CMakeLists.txt +++ b/src/czar/CMakeLists.txt @@ -3,12 +3,16 @@ add_library(czar SHARED) target_sources(czar PRIVATE ChttpModule.cc Czar.cc + CzarChunkMap.cc + CzarRegistry.cc CzarThreads.cc HttpCzarIngestCsvModule.cc HttpCzarIngestModuleBase.cc HttpCzarIngestModule.cc HttpCzarQueryModule.cc HttpCzarSvc.cc + HttpCzarWorkerModule.cc + HttpModule.cc HttpMonitorModule.cc HttpSvc.cc MessageTable.cc @@ -63,3 +67,31 @@ endfunction() czar_utils( qserv-czar-http ) + +function(czar_tests) + foreach(TEST IN ITEMS ${ARGV}) + add_executable(${TEST} ${TEST}.cc) + target_link_libraries(${TEST} PUBLIC + cconfig + ccontrol + czar + global + mysql + parser + qana + qdisp + qproc + qserv_meta + query + rproc + sql + Boost::unit_test_framework + ) + add_test(NAME ${TEST} COMMAND ${TEST}) + endforeach() +endfunction() + +czar_tests( + testCzar +) + diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index bf44d539bb..814c6ee826 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -41,13 +41,16 @@ #include "ccontrol/UserQueryResources.h" #include "ccontrol/UserQuerySelect.h" #include "ccontrol/UserQueryType.h" +#include "czar/CzarChunkMap.h" #include "czar/CzarErrors.h" #include "czar/CzarThreads.h" #include "czar/HttpSvc.h" #include "czar/MessageTable.h" +#include "czar/CzarRegistry.h" #include "global/LogContext.h" #include "proto/worker.pb.h" #include "qdisp/CzarStats.h" +#include "qdisp/Executive.h" #include "qdisp/QdispPool.h" #include "qdisp/SharedResources.h" #include "qproc/DatabaseModels.h" @@ -72,13 +75,6 @@ extern XrdSsiProvider* XrdSsiProviderClient; namespace { -string const createAsyncResultTmpl( - "CREATE TABLE IF NOT EXISTS %1% " - "(jobId BIGINT, resultLocation VARCHAR(1024))" - "ENGINE=MEMORY;" - "INSERT INTO %1% (jobId, resultLocation) " - "VALUES (%2%, '%3%')"); - LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.Czar"); } // anonymous namespace @@ -92,9 +88,63 @@ Czar::Ptr Czar::createCzar(string const& configFilePath, string const& czarName) return _czar; } +void Czar::_monitor() { + string const funcN("Czar::_monitor"); + while (_monitorLoop) { + this_thread::sleep_for(_monitorSleepTime); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start0"); + + /// Check database for changes in worker chunk assignments and aliveness + _czarFamilyMap->read(); + + // TODO:UJ If there were changes in `_czarFamilyMap`, see if any + // workers went down. If any did, `_unassign` all Jobs in UberJobs + // for the downed workers. The `_unassigned` Jobs should get + // reassigned in the next section `assignJobsToUberJobs`. + + /// Create new UberJobs (if possible) for all jobs that are + /// unassigned for any reason. + map> execMap; + { + // Make a copy of all valid Executives + lock_guard execMapLock(_executiveMapMtx); + auto iter = _executiveMap.begin(); + while (iter != _executiveMap.end()) { + auto qIdKey = iter->first; + shared_ptr exec = iter->second.lock(); + if (exec == nullptr) { + iter = _executiveMap.erase(iter); + } else { + execMap[qIdKey] = exec; + ++iter; + } + } + } + // Use the copy to create new UberJobs as needed + for (auto&& [qIdKey, execVal] : execMap) { + execVal->assignJobsToUberJobs(); + } + + // TODO:UJ Maybe get missing results from workers. + // This would be files that workers sent messages to the czar to + // collect, but there was a communication problem and the czar didn't get the message + // or didn't collect the file. to retrieve complete files that haven't been + // collected. + // Basically, is there a reasonable way to check that all UberJobs are being handled + // and nothing has fallen through the cracks? + + // TODO:UJ Maybe send a list of cancelled and completed queries to the workers? + // How long should queryId's remain on this list? + } +} + // Constructors Czar::Czar(string const& configFilePath, string const& czarName) - : _czarName(czarName), _czarConfig(cconfig::CzarConfig::create(configFilePath, czarName)) { + : _czarName(czarName), + _czarConfig(cconfig::CzarConfig::create(configFilePath, czarName)), + _idCounter(), + _uqFactory(), + _clientToQuery() { // set id counter to milliseconds since the epoch, mod 1 year. struct timeval tv; gettimeofday(&tv, nullptr); @@ -111,6 +161,12 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->czarId); + try { + _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + } catch (ChunkMapException const& exc) { + LOGS(_log, LOG_LVL_WARN, string(__func__) + " failed to create CzarChunkMap " + exc.what()); + } + // Tell workers to cancel any queries that were submitted before this restart of Czar. // Figure out which query (if any) was recorded in Czar database before the restart. // The id will be used as the high-watermark for queries that need to be cancelled. @@ -183,6 +239,18 @@ Czar::Czar(string const& configFilePath, string const& czarName) startGarbageCollectAsync(_czarConfig); startGarbageCollectInProgress(_czarConfig, _uqFactory->userQuerySharedResources()->czarId, _uqFactory->userQuerySharedResources()->queryMetadata); + _czarRegistry = CzarRegistry::create(_czarConfig); + + // Start the monitor thread + thread monitorThrd(&Czar::_monitor, this); + _monitorThrd = move(monitorThrd); +} + +Czar::~Czar() { + LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar()"); + _monitorLoop = false; + _monitorThrd.join(); + LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar() end"); } SubmitResult Czar::submitQuery(string const& query, map const& hints) { @@ -252,6 +320,7 @@ SubmitResult Czar::submitQuery(string const& query, map const& h // spawn background thread to wait until query finishes to unlock, // note that lambda stores copies of uq and msgTable. auto finalizer = [uq, msgTable]() mutable { + string qidstr = to_string(uq->getQueryId()); // Add logging context with query ID QSERV_LOGCONTEXT_QUERY(uq->getQueryId()); LOGS(_log, LOG_LVL_DEBUG, "submitting new query"); @@ -265,6 +334,7 @@ SubmitResult Czar::submitQuery(string const& query, map const& h // will likely hang because table may still be locked. LOGS(_log, LOG_LVL_ERROR, "Query finalization failed (client likely hangs): " << exc.what()); } + uq.reset(); }; LOGS(_log, LOG_LVL_DEBUG, "starting finalizer thread for query"); thread finalThread(finalizer); @@ -425,8 +495,15 @@ void Czar::_makeAsyncResult(string const& asyncResultTable, QueryId queryId, str throw exc; } + string const createAsyncResultTmpl( + "CREATE TABLE IF NOT EXISTS %1% " + "(jobId BIGINT, resultLocation VARCHAR(1024))" + "ENGINE=MEMORY;" + "INSERT INTO %1% (jobId, resultLocation) " + "VALUES (%2%, '%3%')"); + string query = - (boost::format(::createAsyncResultTmpl) % asyncResultTable % queryId % resultLocEscaped).str(); + (boost::format(createAsyncResultTmpl) % asyncResultTable % queryId % resultLocEscaped).str(); if (not sqlConn->runQuery(query, sqlErr)) { SqlError exc(ERR_LOC, "Failure creating async result table", sqlErr); @@ -562,4 +639,22 @@ QueryId Czar::_lastQueryIdBeforeRestart() const { return stoull(queryIdStr); } +void Czar::insertExecutive(QueryId qId, std::shared_ptr const& execPtr) { + lock_guard lgMap(_executiveMapMtx); + _executiveMap[qId] = execPtr; +} + +std::shared_ptr Czar::getExecutiveFromMap(QueryId qId) { + lock_guard lgMap(_executiveMapMtx); + auto iter = _executiveMap.find(qId); + if (iter == _executiveMap.end()) { + return nullptr; + } + std::shared_ptr exec = iter->second.lock(); + if (exec == nullptr) { + _executiveMap.erase(iter); + } + return exec; +} + } // namespace lsst::qserv::czar diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 6c7917c776..2e7ec41822 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -58,8 +58,15 @@ namespace lsst::qserv::util { class FileMonitor; } // namespace lsst::qserv::util +namespace lsst::qserv::qdisp { +class Executive; +} // namespace lsst::qserv::qdisp + namespace lsst::qserv::czar { +class CzarFamilyMap; +class CzarRegistry; + /// @addtogroup czar /** @@ -73,6 +80,7 @@ class Czar { Czar(Czar const&) = delete; Czar& operator=(Czar const&) = delete; + ~Czar(); /** * Submit query for execution. @@ -121,6 +129,16 @@ class Czar { /// @return The reconstructed info for the query SubmitResult getQueryInfo(QueryId queryId) const; + std::shared_ptr getCzarFamilyMap() const { return _czarFamilyMap; } + + std::shared_ptr getCzarRegistry() const { return _czarRegistry; } + + /// Add an Executive to the map of executives. + void insertExecutive(QueryId qId, std::shared_ptr const& execPtr); + + /// Get the executive associated with `qId`, this may be nullptr. + std::shared_ptr getExecutiveFromMap(QueryId qId); + private: /// Private constructor for singleton. Czar(std::string const& configFilePath, std::string const& czarName); @@ -140,6 +158,9 @@ class Czar { /// @return An identifier of the last query that was recorded in the query metadata table QueryId _lastQueryIdBeforeRestart() const; + /// Periodically check for system changes and use those changes to try to finish queries. + void _monitor(); + static Ptr _czar; ///< Pointer to single instance of the Czar. // combines client name (ID) and its thread ID into one unique ID @@ -168,6 +189,23 @@ class Czar { /// The HTTP server processing Czar management requests. std::shared_ptr _controlHttpSvc; + + /// Map of which chunks on which workers and shared scan order. + std::shared_ptr _czarFamilyMap; + + /// Connection to the registry to register the czar and get worker contact information. + std::shared_ptr _czarRegistry; + + std::mutex _executiveMapMtx; ///< protects _executiveMap + std::map> + _executiveMap; ///< Map of executives for queries in progress. + + std::thread _monitorThrd; ///< Thread to run the _monitor() + + /// Set to false on system shutdown to stop _monitorThrd. + std::atomic _monitorLoop{true}; + std::chrono::milliseconds _monitorSleepTime{ + 15000}; ///< Wait time between checks. TODO:UJ set from config }; } // namespace lsst::qserv::czar diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc new file mode 100644 index 0000000000..2570b3dc9a --- /dev/null +++ b/src/czar/CzarChunkMap.cc @@ -0,0 +1,418 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "czar/CzarChunkMap.h" + +// System headers +#include + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "qmeta/QMeta.h" +#include "czar/Czar.h" +#include "czar/CzarRegistry.h" +#include "qmeta/Exceptions.h" +#include "util/Bug.h" +#include "util/TimeUtils.h" + +using namespace std; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarChunkMap"); +} // namespace + +namespace lsst::qserv::czar { + +CzarChunkMap::CzarChunkMap() {} + +CzarChunkMap::~CzarChunkMap() { LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); } + +void CzarChunkMap::calcChunkMap(ChunkMap const& chunkMap, ChunkVector& chunksSortedBySize) { + // Calculate total bytes for all chunks. + for (auto&& [chunkIdNum, chunkData] : chunkMap) { + chunkData->_calcTotalBytes(); + chunksSortedBySize.push_back(chunkData); + } + + sortChunks(chunksSortedBySize); +} + +void CzarChunkMap::sortChunks(std::vector& chunksSortedBySize) { + /// Return true if a->_totalBytes > b->_totalBytes + auto sortBySizeDesc = [](ChunkData::Ptr const& a, ChunkData::Ptr const& b) { + if (b == nullptr && a != nullptr) return true; + if (a == nullptr) return false; + return a->_totalBytes > b->_totalBytes; + }; + + std::sort(chunksSortedBySize.begin(), chunksSortedBySize.end(), sortBySizeDesc); +} + +void CzarChunkMap::verify() { + auto&& wcMap = *_workerChunkMap; + auto&& chunkMap = *_chunkMap; + // Use a set to prevent duplicate ids caused by replication levels > 1. + set allChunkIds; + int errorCount = 0; + for (auto const& [wkr, wkrData] : wcMap) { + for (auto const& [chunkId, chunkData] : wkrData->_chunkDataMap) { + allChunkIds.insert(chunkId); + } + } + + for (auto const& [chunkId, chunkDataPtr] : chunkMap) { + if (chunkDataPtr == nullptr) { + LOGS(_log, LOG_LVL_ERROR, " chunkId=" << chunkId << " had nullptr"); + ++errorCount; + continue; + } + auto primeScanWkr = chunkDataPtr->_primaryScanWorker.lock(); + if (primeScanWkr == nullptr) { + LOGS(_log, LOG_LVL_ERROR, " chunkId=" << chunkId << " missing primaryScanWorker"); + ++errorCount; + continue; + } + if (primeScanWkr->_sharedScanChunkMap.find(chunkId) == primeScanWkr->_sharedScanChunkMap.end()) { + LOGS(_log, LOG_LVL_ERROR, + " chunkId=" << chunkId << " should have been (and was not) in the sharedScanChunkMap for " + << primeScanWkr->_workerId); + ++errorCount; + continue; + } + auto iter = allChunkIds.find(chunkId); + if (iter != allChunkIds.end()) { + allChunkIds.erase(iter); + } else { + LOGS(_log, LOG_LVL_ERROR, " chunkId=" << chunkId << " chunkId was not in allChunks list"); + ++errorCount; + continue; + } + } + + auto missing = allChunkIds.size(); + if (missing > 0) { + string allMissingIds; + for (auto const& cId : allChunkIds) { + allMissingIds += to_string(cId) + ","; + } + LOGS(_log, LOG_LVL_ERROR, + " There were " << missing << " missing chunks from the scan list " << allMissingIds); + ++errorCount; + } + + if (errorCount > 0) { + // TODO:UJ There may be an argument to keep the new maps even if there are problems + // with them. For current testing, it's probably best to leave it how it is so that + // it's easier to isolate problems. + throw ChunkMapException(ERR_LOC, "verification failed with " + to_string(errorCount) + " errors"); + } +} + +string CzarChunkMap::dumpChunkMap(ChunkMap const& chunkMap) { + stringstream os; + os << "ChunkMap{"; + for (auto const& [cId, cDataPtr] : chunkMap) { + os << "(cId=" << cId << ":"; + os << ((cDataPtr == nullptr) ? "null" : cDataPtr->dump()) << ")"; + } + os << "}"; + return os.str(); +} + +string CzarChunkMap::dumpWorkerChunkMap(WorkerChunkMap const& wcMap) { + stringstream os; + os << "WorkerChunkMap{"; + for (auto const& [wId, wDataPtr] : wcMap) { + os << "(wId=" << wId << ":"; + os << ((wDataPtr == nullptr) ? "null" : wDataPtr->dump()) << ")"; + } + os << "}"; + return os.str(); +} + +void CzarChunkMap::ChunkData::_calcTotalBytes() { + _totalBytes = 0; + for (auto const& [key, val] : _dbTableMap) { + _totalBytes += val; + } +} + +void CzarChunkMap::ChunkData::addToWorkerHasThis(std::shared_ptr const& worker) { + if (worker == nullptr) { + throw ChunkMapException(ERR_LOC, string(__func__) + " worker was null"); + } + + _workerHasThisMap[worker->_workerId] = worker; +} + +std::map> +CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() const { + std::map> newMap = _workerHasThisMap; + return newMap; +} + +void CzarChunkMap::organize() { + auto chunksSortedBySize = make_shared(); + + calcChunkMap(*_chunkMap, *chunksSortedBySize); + + // At this point we have + // - _workerChunkMap has a map of workerData by worker id with each worker having a map of ChunkData + // - _chunkMap has a map of all chunkData by chunk id + // - chunksSortedBySize a list of chunks sorted with largest first. + // From here need to assign shared scan chunk priority + // Go through the chunksSortedBySize list and assign each chunk to worker that has it with the smallest + // totalScanSize. + for (auto&& chunkData : *chunksSortedBySize) { + SizeT smallest = std::numeric_limits::max(); + WorkerChunksData::Ptr smallestWkr = nullptr; + for (auto&& [wkrId, wkrDataWeak] : chunkData->_workerHasThisMap) { + auto wkrData = wkrDataWeak.lock(); + if (wkrData == nullptr) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " unexpected null weak ptr for " << wkrId); + continue; // maybe the next one will be okay. + } + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " wkrId=" << wkrData << " tsz=" << wkrData->_sharedScanTotalSize + << " smallest=" << smallest); + if (wkrData->_sharedScanTotalSize < smallest) { + smallestWkr = wkrData; + smallest = smallestWkr->_sharedScanTotalSize; + } + } + if (smallestWkr == nullptr) { + throw ChunkMapException(ERR_LOC, string(__func__) + " no smallesWkr found for chunk=" + + to_string(chunkData->_chunkId)); + } + smallestWkr->_sharedScanChunkMap[chunkData->_chunkId] = chunkData; + smallestWkr->_sharedScanTotalSize += chunkData->_totalBytes; + chunkData->_primaryScanWorker = smallestWkr; + LOGS(_log, LOG_LVL_DEBUG, + " chunk=" << chunkData->_chunkId << " assigned to scan on " << smallestWkr->_workerId); + } +} + +string CzarChunkMap::ChunkData::dump() const { + stringstream os; + auto primaryWorker = _primaryScanWorker.lock(); + os << "{ChunkData id=" << _chunkId << " totalBytes=" << _totalBytes; + os << " primaryWorker=" << ((primaryWorker == nullptr) ? "null" : primaryWorker->_workerId); + os << " workers{"; + for (auto const& [wId, wData] : _workerHasThisMap) { + os << "(" << wId << ")"; + } + os << "} tables{"; + for (auto const& [dbTbl, sz] : _dbTableMap) { + os << "(" << dbTbl.first << "." << dbTbl.second << " sz=" << sz << ")"; + } + os << "}}"; + return os.str(); +} + +string CzarChunkMap::WorkerChunksData::dump() const { + stringstream os; + os << "{WorkerChunksData id=" << _workerId << " scanTotalSize=" << _sharedScanTotalSize; + os << " chunkDataIds{"; + for (auto const& [chunkId, chunkData] : _chunkDataMap) { + os << "(" << chunkId << ")"; + } + os << "} sharedScanChunks{"; + for (auto const& [chunkId, chunkData] : _sharedScanChunkMap) { + os << "(" << chunkId << ")"; + } + os << "}}"; + return os.str(); +} + +CzarFamilyMap::CzarFamilyMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { + try { + auto mapsSet = _read(); + if (!mapsSet) { + throw ChunkMapException(ERR_LOC, cName(__func__) + " maps were not set in contructor"); + } + } catch (qmeta::QMetaError const& qExc) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not read DB " << qExc.what()); + throw ChunkMapException(ERR_LOC, cName(__func__) + " constructor failed read " + qExc.what()); + } +} + +bool CzarFamilyMap::read() { + bool mapsSet = false; + try { + mapsSet = _read(); + } catch (qmeta::QMetaError const& qExc) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) + " could not read DB " << qExc.what()); + } + return mapsSet; +} + +bool CzarFamilyMap::_read() { + LOGS(_log, LOG_LVL_TRACE, "CzarFamilyMap::_read() start"); + // If replacing the map, this may take a bit of time, but it's probably + // better to wait for new maps if something changed. + std::lock_guard gLock(_familyMapMtx); + qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(_lastUpdateTime); + if (_lastUpdateTime >= qChunkMap.updateTime) { + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " no need to read " + << util::TimeUtils::timePointToDateTimeString(_lastUpdateTime) + << " db=" << util::TimeUtils::timePointToDateTimeString(qChunkMap.updateTime)); + return false; + } + + // Make the new maps. + shared_ptr familyMapPtr = makeNewMaps(qChunkMap); + + verify(familyMapPtr); + + _familyMap = familyMapPtr; + + _lastUpdateTime = qChunkMap.updateTime; + + LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_read() end"); + return true; +} + +std::shared_ptr CzarFamilyMap::makeNewMaps( + qmeta::QMetaChunkMap const& qChunkMap) { + // Create new maps. + std::shared_ptr newFamilyMap = make_shared(); + + // Workers -> Databases map + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " workers.sz=" << qChunkMap.workers.size()); + for (auto const& [workerId, dbs] : qChunkMap.workers) { + // Databases -> Tables map + for (auto const& [dbName, tables] : dbs) { + // Tables -> Chunks map + for (auto const& [tableName, chunks] : tables) { + // vector of ChunkInfo + for (qmeta::QMetaChunkMap::ChunkInfo const& chunkInfo : chunks) { + try { + int64_t chunkNum = chunkInfo.chunk; + CzarChunkMap::SizeT sz = chunkInfo.size; + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << "workerdId=" << workerId << " db=" << dbName << " table=" + << tableName << " chunk=" << chunkNum << " sz=" << sz); + insertIntoMaps(newFamilyMap, workerId, dbName, tableName, chunkNum, sz); + } catch (invalid_argument const& exc) { + throw ChunkMapException( + ERR_LOC, cName(__func__) + " invalid_argument workerdId=" + workerId + + " db=" + dbName + " table=" + tableName + + " chunk=" + to_string(chunkInfo.chunk) + " " + exc.what()); + } catch (out_of_range const& exc) { + throw ChunkMapException( + ERR_LOC, cName(__func__) + " out_of_range workerdId=" + workerId + + " db=" + dbName + " table=" + tableName + + " chunk=" + to_string(chunkInfo.chunk) + " " + exc.what()); + } + } + } + } + } + + // this needs to be done for each CzarChunkMap in the family map. + for (auto&& [familyName, chunkMapPtr] : *newFamilyMap) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " working on " << familyName); + chunkMapPtr->organize(); + } + + return newFamilyMap; +} + +void CzarFamilyMap::insertIntoMaps(std::shared_ptr const& newFamilyMap, string const& workerId, + string const& dbName, string const& tableName, int64_t chunkIdNum, + CzarChunkMap::SizeT sz) { + // Get the CzarChunkMap for this family + auto familyName = getFamilyNameFromDbName(dbName); + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " familyInsrt{w=" << workerId << " fN=" << familyName << " dbN=" << dbName + << " tblN=" << tableName << " chunk=" << chunkIdNum << " sz=" << sz << "}"); + auto& nfMap = *newFamilyMap; + CzarChunkMap::Ptr czarChunkMap; + auto familyIter = nfMap.find(familyName); + if (familyIter == nfMap.end()) { + czarChunkMap = CzarChunkMap::Ptr(new CzarChunkMap()); + nfMap[familyName] = czarChunkMap; + } else { + czarChunkMap = familyIter->second; + } + + auto [chunkMapPtr, wcMapPtr] = czarChunkMap->_getMaps(); + + CzarChunkMap::WorkerChunkMap& wcMap = *wcMapPtr; + CzarChunkMap::ChunkMap& chunkMap = *chunkMapPtr; + + // Get or make the worker entry + CzarChunkMap::WorkerChunksData::Ptr workerChunksData; + auto iterWC = wcMap.find(workerId); + if (iterWC == wcMap.end()) { + workerChunksData = CzarChunkMap::WorkerChunksData::Ptr(new CzarChunkMap::WorkerChunksData(workerId)); + wcMap[workerId] = workerChunksData; + } else { + workerChunksData = iterWC->second; + } + + // Get or make the ChunkData entry in chunkMap + CzarChunkMap::ChunkData::Ptr chunkData; + auto iterChunkData = chunkMap.find(chunkIdNum); + if (iterChunkData == chunkMap.end()) { + chunkData = CzarChunkMap::ChunkData::Ptr(new CzarChunkMap::ChunkData(chunkIdNum)); + chunkMap[chunkIdNum] = chunkData; + } else { + chunkData = iterChunkData->second; + } + + // Set or verify the table information + auto iterDT = chunkData->_dbTableMap.find({dbName, tableName}); + if (iterDT == chunkData->_dbTableMap.end()) { + // doesn't exist so set it up + chunkData->_dbTableMap[{dbName, tableName}] = sz; + } else { + // Verify that it matches other data + auto const& dbTbl = iterDT->first; + auto tblSz = iterDT->second; + auto const& dbN = dbTbl.first; + auto const& tblN = dbTbl.second; + if (dbName != dbN || tblN != tableName || tblSz != sz) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " data mismatch for " << dbName << "." << tableName << "=" << sz << " vs " + << dbN << "." << tblN << "=" << tblSz); + } + } + + // Link WorkerData the single chunkData instance for the chunkId + workerChunksData->_chunkDataMap[chunkIdNum] = chunkData; + + // Add worker to the list of workers containing the chunk. + chunkData->addToWorkerHasThis(workerChunksData); +} + +void CzarFamilyMap::verify(std::shared_ptr const& familyMap) { + for (auto&& [familyName, czarChunkMapPtr] : *familyMap) { + czarChunkMapPtr->verify(); + } +} + +} // namespace lsst::qserv::czar diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h new file mode 100644 index 0000000000..b6f69d38b2 --- /dev/null +++ b/src/czar/CzarChunkMap.h @@ -0,0 +1,324 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +#ifndef LSST_QSERV_CZAR_CZARCHUNKMAP_H +#define LSST_QSERV_CZAR_CZARCHUNKMAP_H + +// System headers +#include +#include +#include +#include +#include +#include +#include + +// Qserv headers +#include "global/clock_defs.h" +#include "util/Issue.h" + +namespace lsst::qserv::qmeta { +class QMeta; +struct QMetaChunkMap; +} // namespace lsst::qserv::qmeta + +namespace lsst::qserv::czar { + +class CzarFamilyMap; + +class ChunkMapException : public util::Issue { +public: + ChunkMapException(Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} +}; + +/// The czar is expected to heavily use the +/// `getMaps() -> WorkerChunkMap -> getSharedScanChunkMap()` +/// to send jobs to workers, as that gets an ordered list of all chunks +/// the worker should handle during a shared scan. +/// `getMaps() -> ChunkMap` is expected to be more useful if there is a +/// failure and a chunk query needs to go to a different worker. +/// +/// Workers failing or new workers being added is expected to be a rare event. +/// The current algorithm to split chunks between the workers tries to split +/// the work evenly. However, if a new worker is added, it's likely that +/// the new distribution of chunks for shared scans will put the chunks on +/// different workers than previously, which in turn will result in the system +/// being less efficient until all the old scans are complete. If workers +/// being added or removed from the system becomes frequent, the algorithm should +/// probably change to try to maintain some chunk location consistency once +/// the system is up. +class CzarChunkMap { +public: + using Ptr = std::shared_ptr; + using SizeT = uint64_t; + + CzarChunkMap(CzarChunkMap const&) = delete; + CzarChunkMap& operator=(CzarChunkMap const&) = delete; + + // static Ptr create(std::shared_ptr const& qmeta) { return Ptr(new CzarChunkMap(qmeta)); } + static Ptr create() { return Ptr(new CzarChunkMap()); } + + ~CzarChunkMap(); + + class WorkerChunksData; + + /// Essentially a structure for storing data about which tables and workers are associated with this + /// chunk. + class ChunkData { + public: + using Ptr = std::shared_ptr; + ChunkData(int chunkId_) : _chunkId(chunkId_) {} + + int64_t getChunkId() const { return _chunkId; } + + SizeT getTotalBytes() const { return _totalBytes; } + + std::weak_ptr getPrimaryScanWorker() const { return _primaryScanWorker; } + + /// Add `worker` to the `_workerHasThisMap` to indicate that worker has a copy + /// of this chunk. + void addToWorkerHasThis(std::shared_ptr const& worker); + + /// Return a copy of _workerHasThisMap. + std::map> getWorkerHasThisMapCopy() const; + + std::string dump() const; + + friend CzarChunkMap; + friend CzarFamilyMap; + + private: + int64_t const _chunkId; ///< The Id number for this chunk. + SizeT _totalBytes = 0; ///< The total number of bytes used by all tables in this chunk. + std::weak_ptr _primaryScanWorker; ///< The worker to be used to shared scans. + + /// Key is databaseName+tableName, value is size in bytes. + std::map, SizeT> _dbTableMap; + + /// Map of workers that have this chunk + std::map> _workerHasThisMap; + + /// Add up the bytes in each table for this chunk to get `_totalBytes` + void _calcTotalBytes(); + }; + + /// Essentially a structure for storing which chunks are associated with a worker. + class WorkerChunksData { + public: + using Ptr = std::shared_ptr; + WorkerChunksData(std::string const& workerId) : _workerId(workerId) {} + + /// Return the worker's id string. + std::string const& getWorkerId() const { return _workerId; } + + /// Return the number of bytes contained in all chunks/tables to be + /// accessed in a full table scan on this worker. + SizeT getSharedScanTotalSize() const { return _sharedScanTotalSize; } + + /// Return a reference to `_sharedScanChunkMap`. A copy of the pointer + /// to this class (or the containing map) should be held to ensure the reference. + std::map const& getSharedScanChunkMap() const { return _sharedScanChunkMap; } + + std::string dump() const; + + friend CzarChunkMap; + friend CzarFamilyMap; + + private: + std::string const _workerId; + + /// Map of all chunks found on the worker where key is chunkId + std::map _chunkDataMap; + + /// Map of chunks this worker will handle during shared scans. + /// Since scans are done in order of chunk id numbers, it helps + /// to have this in chunk id number order. + /// At some point, thus should be sent to workers so they + /// can make more accurate time estimates for chunk completion. + std::map _sharedScanChunkMap; + + /// The total size (in bytes) of all chunks on this worker that + /// are to be used in shared scans. + SizeT _sharedScanTotalSize = 0; + }; + + using WorkerChunkMap = std::map; + using ChunkMap = std::map; + using ChunkVector = std::vector; + + /// Sort the chunks in `chunksSortedBySize` in descending order by total size in bytes. + static void sortChunks(ChunkVector& chunksSortedBySize); + + /// Calculate the total bytes in each chunk and then sort the resulting ChunkVector by chunk size, + /// descending. + static void calcChunkMap(ChunkMap const& chunkMap, ChunkVector& chunksSortedBySize); + + /// Verify that all chunks belong to at least one worker and that all chunks are represented in shared + /// scans. + /// @throws ChunkMapException + void verify(); + + static std::string dumpChunkMap(ChunkMap const& chunkMap); + + static std::string dumpWorkerChunkMap(WorkerChunkMap const& wcMap); + + /// Return shared pointers to `_chunkMap` and `_workerChunkMap`, which should be held until + /// finished with the data. + std::pair, + std::shared_ptr> + getMaps() const { + std::lock_guard lck(_mapMtx); + return {_chunkMap, _workerChunkMap}; + } + + /// Use the information from the registry to `organize` `_chunkMap` and `_workerChunkMap` + /// into their expected formats. + void organize(); + +private: + CzarChunkMap(); + + /// Return shared pointers to `_chunkMap` and `_workerChunkMap`, which should be held until + /// finished with the data. + std::pair, std::shared_ptr> + _getMaps() const { + std::lock_guard lck(_mapMtx); + return {_chunkMap, _workerChunkMap}; + } + + /// Map of all workers and which chunks they contain. + std::shared_ptr _workerChunkMap{new WorkerChunkMap()}; + + /// Map of all chunks in the system with chunkId number as the key and the values contain + /// information about the tables in those chunks and which worker is responsible for + /// handling the chunk in a shared scan. + std::shared_ptr _chunkMap{new ChunkMap()}; + + mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap (TODO:UJ may not be needed anymore) + + friend CzarFamilyMap; +}; + +/// This class is used to organize worker chunk table information so that it +/// can be used to send jobs to the appropriate worker and inform workers +/// what chunks they can expect to handle in shared scans, focusing at the +/// family level. +/// The data for the maps is provided by the Replicator and stored in the +/// QMeta database. +/// When the data is changed, there is a timestamp that is updated, which +/// will cause new maps to be made by this class. +/// +/// The maps generated should be treated as constant objects stored with +/// shared pointers. As such, it should be possible for numerous threads +/// to use each map simultaneously provided they have their own pointers +/// to the maps. +/// The pointers to the maps are mutex protected to safely allow map updates. +// +// TODO:UJ move this to its own header file. +// +// TODO:UJ Currently, each family only has one database and they share a name. +// Once a table mapping databases to families is available, it needs to be +// used to map databases to families in this class. +class CzarFamilyMap { +public: + using Ptr = std::shared_ptr; + typedef std::map FamilyMapType; + typedef std::map DbNameToFamilyNameType; + + static Ptr create(std::shared_ptr const& qmeta) { return Ptr(new CzarFamilyMap(qmeta)); } + + CzarFamilyMap() = delete; + CzarFamilyMap(CzarFamilyMap const&) = delete; + CzarFamilyMap& operator=(CzarFamilyMap const&) = delete; + + ~CzarFamilyMap() = default; + + /// For unit testing only + /// @param dbNameToFamilyNameType - valid map of db to family name for the unit test. + // TODO::UJ define member instance for `_dbNameToFamilyName` + CzarFamilyMap(std::shared_ptr const& dbNameToFamilyName) {} + + std::string cName(const char* fName) const { + return std::string("CzarFamilyMap::") + ((fName == nullptr) ? "?" : fName); + } + + /// Family names are unknown until a table has been added to the database, so + /// the dbName will be used as the family name until the table exists. + std::string getFamilyNameFromDbName(std::string const& dbName) const { + // TODO:UJ use a member instance of std::shared_ptr + // once info is available in QMeta. + return dbName; + } + + /// Return the chunk map for the database `dbName` + CzarChunkMap::Ptr getChunkMap(std::string const& dbName) const { + auto familyName = getFamilyNameFromDbName(dbName); + return _getChunkMap(familyName); + } + + /// Read the registry information from the database, if not already set. + bool read(); + + /// Make a new FamilyMapType map including ChunkMap and WorkerChunkMap from the data + /// in `qChunkMap`. Each family has its own ChunkMap and WorkerChunkMap. + std::shared_ptr makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); + + /// Insert the new element described by the parameters into the `newFamilyMap` as appropriate. + void insertIntoMaps(std::shared_ptr const& newFamilyMap, std::string const& workerId, + std::string const& dbName, std::string const& tableName, int64_t chunkIdNum, + CzarChunkMap::SizeT sz); + + /// Verify the `familyMap` does not have errors. + static void verify(std::shared_ptr const& familyMap); + +private: + /// Try to `_read` values for maps from `qmeta`. + CzarFamilyMap(std::shared_ptr const& qmeta); + + /// Read the registry information from the database, stopping if + /// it hasn't been updated. + // TODO:UJ add a changed timestamp (similar to the existing updated timestamp) + // to the registry database and only update when changed. + bool _read(); + + /// Return the chunk map for the `familyName` + CzarChunkMap::Ptr _getChunkMap(std::string const& familyName) const { + std::lock_guard familyLock(_familyMapMtx); + auto iter = _familyMap->find(familyName); + if (iter == _familyMap->end()) { + return nullptr; + } + return iter->second; + } + + std::shared_ptr _qmeta; ///< Database connection to collect json worker list. + + /// The last time the maps were updated with information from the replicator. + TIMEPOINT _lastUpdateTime; // initialized to 0; + + std::shared_ptr _familyMap{new FamilyMapType()}; + mutable std::mutex _familyMapMtx; ///< protects _familyMap, _timeStamp, and _qmeta. +}; + +} // namespace lsst::qserv::czar + +#endif // LSST_QSERV_CZAR_CZARCHUNKMAP_H diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc new file mode 100644 index 0000000000..f5abfcaba6 --- /dev/null +++ b/src/czar/CzarRegistry.cc @@ -0,0 +1,198 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "czar/CzarRegistry.h" + +// System headers +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// Qserv headers +#include "cconfig/CzarConfig.h" +#include "czar/CzarChunkMap.h" +#include "czar/Czar.h" +#include "http/Client.h" +#include "http/Method.h" +#include "util/common.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarRegistry"); +} // namespace + +namespace lsst::qserv::czar { + +CzarRegistry::CzarRegistry(std::shared_ptr const& czarConfig) : _czarConfig(czarConfig) { + // Begin periodically updating worker's status in the Replication System's registry. + // This will continue until the application gets terminated. + thread registryUpdateThread(&CzarRegistry::_registryUpdateLoop, this); + _czarHeartbeatThrd = move(registryUpdateThread); + + thread registryWorkerUpdateThread(&CzarRegistry::_registryWorkerInfoLoop, this); + _czarWorkerInfoThrd = move(registryWorkerUpdateThread); +} + +CzarRegistry::~CzarRegistry() { + _loop = false; + if (_czarHeartbeatThrd.joinable()) { + _czarHeartbeatThrd.join(); + } + if (_czarWorkerInfoThrd.joinable()) { + _czarWorkerInfoThrd.join(); + } +} + +void CzarRegistry::_registryUpdateLoop() { + auto const method = http::Method::POST; + string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + + to_string(_czarConfig->replicationRegistryPort()) + "/czar"; + vector const headers = {"Content-Type: application/json"}; + json const request = json::object({{"instance_id", _czarConfig->replicationInstanceId()}, + {"auth_key", _czarConfig->replicationAuthKey()}, + {"czar", + {{"name", _czarConfig->name()}, + {"id", _czarConfig->id()}, + {"management-port", _czarConfig->replicationHttpPort()}, + {"management-host-name", util::get_current_host_fqdn()}}}}); + string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; + LOGS(_log, LOG_LVL_TRACE, + __func__ << " czarPost url=" << url << " request=" << request.dump() << " headers=" << headers[0]); + http::Client client(method, url, request.dump(), headers); + while (_loop) { + LOGS(_log, LOG_LVL_TRACE, + __func__ << " loop url=" << url << " request=" << request.dump() << " headers=" << headers[0]); + try { + json const response = client.readAsJson(); + if (0 == response.at("success").get()) { + string const error = response.at("error").get(); + LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); + // TODO: Is there a better thing to do than just log this here? + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + } + this_thread::sleep_for(chrono::seconds(max(1U, _czarConfig->replicationRegistryHearbeatIvalSec()))); + } +} + +void CzarRegistry::_registryWorkerInfoLoop() { + // Get worker information from the registry + vector const headers; + auto const method = http::Method::GET; + string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + + to_string(_czarConfig->replicationRegistryPort()) + + "/services?instance_id=" + _czarConfig->replicationInstanceId(); + string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; + LOGS(_log, LOG_LVL_TRACE, __func__ << " url=" << url); + http::Client client(method, url, string(), headers); + while (_loop) { + try { + json const response = client.readAsJson(); + if (0 == response.at("success").get()) { + string const error = response.at("error").get(); + LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); + // TODO: Is there a better thing to do than just log this here? + } else { + WorkerContactMapPtr wMap = _buildMapFromJson(response); + // Compare the new map to the existing map and replace if different. + { + lock_guard lck(_mapMtx); + if (wMap != nullptr && !_compareMap(*wMap)) { + _contactMap = wMap; + _latestUpdate = CLOCK::now(); + } + } + } + LOGS(_log, LOG_LVL_TRACE, __func__ << " resp=" << response); + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + } + this_thread::sleep_for(chrono::seconds(15)); + } +} + +CzarRegistry::WorkerContactMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { + auto const& jsServices = response.at("services"); + auto const& jsWorkers = jsServices.at("workers"); + auto wMap = WorkerContactMapPtr(new WorkerContactMap()); + for (auto const& [key, value] : jsWorkers.items()) { + auto const& jsQserv = value.at("qserv"); + LOGS(_log, LOG_LVL_DEBUG, __func__ << " key=" << key << " jsQ=" << jsQserv); + string wHost = jsQserv.at("host-addr").get(); + string wManagementHost = jsQserv.at("management-host-name").get(); + int wPort = jsQserv.at("management-port").get(); + uint64_t updateTimeInt = jsQserv.at("update-time-ms").get(); + TIMEPOINT updateTime = TIMEPOINT(chrono::milliseconds(updateTimeInt)); + auto wInfo = make_shared(key, wHost, wManagementHost, wPort, updateTime); + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " wHost=" << wHost << " wPort=" << wPort << " updateTime=" << updateTimeInt); + auto iter = wMap->find(key); + if (iter != wMap->end()) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " duplicate key " << key << " in " << response); + if (!wInfo->sameContactInfo(*(iter->second))) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " incongruent key " << key << " in " << response); + return nullptr; + } + // ignore the duplicate, since it matches the previous one. + } else { + wMap->insert({key, wInfo}); + } + } + return wMap; +} + +bool CzarRegistry::_compareMap(WorkerContactMap const& other) const { + if (_contactMap == nullptr) { + // If _contactMap is null, it needs to be replaced. + return false; + } + if (other.size() != _contactMap->size()) { + return false; + } + for (auto const& [key, wInfo] : *_contactMap) { + auto iter = other.find(key); + if (iter == other.end()) { + return false; + } else { + if (!(iter->second->sameContactInfo(*wInfo))) { + return false; + } + } + } + return true; +} + +string CzarRegistry::WorkerContactInfo::dump() const { + stringstream os; + os << "workerContactInfo{" + << "id=" << wId << " host=" << wHost << " mgHost=" << wManagementHost << " port=" << wPort << "}"; + return os.str(); +} + +} // namespace lsst::qserv::czar diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h new file mode 100644 index 0000000000..27d20979cf --- /dev/null +++ b/src/czar/CzarRegistry.h @@ -0,0 +1,137 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_CZAR_CZARREGISTRY_H +#define LSST_QSERV_CZAR_CZARREGISTRY_H + +// System headers +#include +#include +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// Qserv headers +#include "global/clock_defs.h" + +namespace lsst::qserv::cconfig { +class CzarConfig; +} // namespace lsst::qserv::cconfig + +namespace lsst::qserv::czar { + +/// This class connects to the Replication System's Registry to register this czar and get +/// worker contact information. +/// The assumptions going forward are that the CzarChunkMap provides the real location of +/// where all chunks are located and any workers in that map that are missing from this +/// map are just temporary communications problems. A real prolonged failure of a worker +/// will result in a new CzarChunkMap being created. As such, problems with missing +/// worker contact information will be handled in Job creation +/// in UserQueryFactory::newUserQuery and will be treated in similar manner as not being +/// able to contact a worker. +/// +/// There really shouldn't be communications problems, but there are, the best course of +/// action would probably be to destroy the first instance of this and create a new one. +/// +class CzarRegistry { +public: + using Ptr = std::shared_ptr; + + /// Return a pointer to a new CzarRegistry object. + static Ptr create(std::shared_ptr const& czarConfig) { + return Ptr(new CzarRegistry(czarConfig)); + } + + ~CzarRegistry(); + + struct WorkerContactInfo { + using Ptr = std::shared_ptr; + + WorkerContactInfo(std::string const& wId_, std::string const& wHost_, + std::string const& wManagementHost_, int wPort_, TIMEPOINT updateTime_) + : wId(wId_), + wHost(wHost_), + wManagementHost(wManagementHost_), + wPort(wPort_), + updateTime(updateTime_) {} + std::string const wId; ///< key + std::string const wHost; ///< "host-addr" entry. + std::string const wManagementHost; ///< "management-host-name" entry. + int const wPort; ///< "management-port" entry. + TIMEPOINT const updateTime; ///< "update-time-ms" entry. + + /// Return true if all members, aside from updateTime, are equal. + bool sameContactInfo(WorkerContactInfo const& other) const { + return (wId == other.wId && wHost == other.wHost && wManagementHost == other.wManagementHost && + wPort == other.wPort); + } + std::string dump() const; + }; + + using WorkerContactMap = std::unordered_map; + using WorkerContactMapPtr = std::shared_ptr; + + /// Return _contactMap, the object that the returned pointer points to is + /// constant and no attempts should be made to change it. + WorkerContactMapPtr getWorkerContactMap() { + std::lock_guard lockG(_mapMtx); + return _contactMap; + } + +private: + CzarRegistry() = delete; + CzarRegistry(std::shared_ptr const& czarConfig); + + /// This function will keep periodically updating Czar's info in the Replication System's Registry + /// until _loop is set to false. + /// Communications problems are logged but ignored. This should probably change. + void _registryUpdateLoop(); + + /// This function collects worker contact information from the Replication System's Registry + /// until _loop is set to false. + /// Communications problems are logged but ignored. This should probably change. + void _registryWorkerInfoLoop(); + + /// Build a new WorkerContactMap from the json `response` + WorkerContactMapPtr _buildMapFromJson(nlohmann::json const& response); + + /// Return true if maps are the same size and all of the elements are the same(). + bool _compareMap(WorkerContactMap const& other) const; + + std::shared_ptr const _czarConfig; ///< Pointer to the CzarConfig. + + std::atomic _loop{true}; ///< Threads will continue to run until this is set false. + std::thread _czarHeartbeatThrd; ///< This thread continually registers this czar with the registry. + std::thread _czarWorkerInfoThrd; ///< This thread continuously collects worker contact information. + + /// Pointer to the map of worker contact information. + WorkerContactMapPtr _contactMap; + TIMEPOINT _latestUpdate; ///< The last time the _contactMap was updated. + std::mutex _mapMtx; /// Protects _contactMap, _latestUpdate. +}; + +} // namespace lsst::qserv::czar + +#endif // LSST_QSERV_CZAR_CZARREGISTRY_H diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc new file mode 100644 index 0000000000..3d179d577f --- /dev/null +++ b/src/czar/HttpCzarWorkerModule.cc @@ -0,0 +1,168 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "czar/HttpCzarWorkerModule.h" + +// System headers +#include +#include + +// Qserv headers +#include "cconfig/CzarConfig.h" +#include "czar/Czar.h" +#include "qdisp/Executive.h" +#include "qdisp/UberJob.h" +#include "global/intTypes.h" +#include "http/Exceptions.h" +#include "http/RequestQuery.h" +#include "util/String.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using json = nlohmann::json; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.HttpCzarWorkerModule"); +} + +namespace lsst::qserv::czar { + +void HttpCzarWorkerModule::process(string const& context, shared_ptr const& req, + shared_ptr const& resp, string const& subModuleName, + http::AuthType const authType) { + HttpCzarWorkerModule module(context, req, resp); + module.execute(subModuleName, authType); +} + +HttpCzarWorkerModule::HttpCzarWorkerModule(string const& context, shared_ptr const& req, + shared_ptr const& resp) + : HttpModule(context, req, resp) {} + +json HttpCzarWorkerModule::executeImpl(string const& subModuleName) { + string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; + debug(func); + enforceCzarName(func); + if (subModuleName == "QUERYJOB-ERROR") + return _queryJobError(); + else if (subModuleName == "QUERYJOB-READY") + return _queryJobReady(); + throw invalid_argument(context() + func + " unsupported sub-module"); +} + +json HttpCzarWorkerModule::_queryJobError() { + debug(__func__); + checkApiVersion(__func__, 34); + LOGS(_log, LOG_LVL_DEBUG, __func__ << " queryJobError json=" << body().objJson); + auto ret = _handleJobError(__func__); + return json::object(); +} + +json HttpCzarWorkerModule::_queryJobReady() { + debug(__func__); + checkApiVersion(__func__, 34); + LOGS(_log, LOG_LVL_DEBUG, __func__ << " queryJobReady json=" << body().objJson); + auto ret = _handleJobReady(__func__); + return ret; +} + +json HttpCzarWorkerModule::_handleJobError(string const& func) { + // Metadata-only responses for the file-based protocol should not have any data + + // Parse and verify the json message and then kill the UberJob. + json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + string const targetWorkerId = body().required("workerid"); + string const czarName = body().required("czar"); + qmeta::CzarId const czarId = body().required("czarid"); + QueryId const queryId = body().required("queryid"); + UberJobId const uberJobId = body().required("uberjobid"); + int const errorCode = body().required("errorCode"); + string const errorMsg = body().required("errorMsg"); + + // Find UberJob + qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); + if (exec == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No executive for qid=") + + to_string(queryId) + " czar=" + to_string(czarId)); + } + qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); + if (uj == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No UberJob for qid=") + + to_string(queryId) + " ujId=" + to_string(uberJobId) + + " czar=" + to_string(czarId)); + } + + auto importRes = uj->workerError(errorCode, errorMsg); + jsRet = importRes; + + } catch (std::invalid_argument const& iaEx) { + LOGS(_log, LOG_LVL_ERROR, + "HttpCzarWorkerModule::_handleJobError received " << iaEx.what() << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; + } + return jsRet; +} + +json HttpCzarWorkerModule::_handleJobReady(string const& func) { + // Metadata-only responses for the file-based protocol should not have any data + + // Parse and verify the json message and then have the uberjob import the file. + json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + string const targetWorkerId = body().required("workerid"); + string const czarName = body().required("czar"); + qmeta::CzarId const czarId = body().required("czarid"); + QueryId const queryId = body().required("queryid"); + UberJobId const uberJobId = body().required("uberjobid"); + string const fileUrl = body().required("fileUrl"); + uint64_t const rowCount = body().required("rowCount"); + uint64_t const fileSize = body().required("fileSize"); + + // Find UberJob + qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); + if (exec == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + + to_string(queryId) + " czar=" + to_string(czarId)); + } + qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); + if (uj == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No UberJob for qid=") + + to_string(queryId) + " ujId=" + to_string(uberJobId) + + " czar=" + to_string(czarId)); + } + + auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); + jsRet = importRes; + + } catch (std::invalid_argument const& iaEx) { + LOGS(_log, LOG_LVL_ERROR, + "HttpCzarWorkerModule::_handleJobReady received " << iaEx.what() << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; + } + return jsRet; +} + +} // namespace lsst::qserv::czar diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h new file mode 100644 index 0000000000..ef0a745093 --- /dev/null +++ b/src/czar/HttpCzarWorkerModule.h @@ -0,0 +1,82 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_CZAR_HTTPCZARWORKERMODULE_H +#define LSST_QSERV_CZAR_HTTPCZARWORKERMODULE_H + +// System headers +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// Qserv headers +#include "czar/HttpModule.h" + +// Forward declarations +namespace lsst::qserv::qhttp { +class Request; +class Response; +} // namespace lsst::qserv::qhttp + +// This header declarations +namespace lsst::qserv::czar { + +/// This class is used to handle messages to this czar from the workers. +class HttpCzarWorkerModule : public czar::HttpModule { +public: + /// @note supported values for parameter 'subModuleName' are: + /// 'QUERYJOB-ERROR' - error in a QUERYJOB + /// 'QUERYJOB-READY' - + /// @throws std::invalid_argument for unknown values of parameter 'subModuleName' + static void process(std::string const& context, std::shared_ptr const& req, + std::shared_ptr const& resp, std::string const& subModuleName, + http::AuthType const authType = http::AuthType::NONE); + + HttpCzarWorkerModule() = delete; + HttpCzarWorkerModule(HttpCzarWorkerModule const&) = delete; + HttpCzarWorkerModule& operator=(HttpCzarWorkerModule const&) = delete; + + ~HttpCzarWorkerModule() final = default; + +protected: + nlohmann::json executeImpl(std::string const& subModuleName) final; + +private: + HttpCzarWorkerModule(std::string const& context, std::shared_ptr const& req, + std::shared_ptr const& resp); + + /// Called to handle message indicating this czar needs to handle an error on a worker. + nlohmann::json _queryJobError(); + + /// Called to indicate an UberJob is ready with data that needs to be collected. + nlohmann::json _queryJobReady(); + + /// Translates the message and calls the Czar to collect the data. + nlohmann::json _handleJobReady(std::string const& func); + + /// Translates the error and calls the Czar to take action. + nlohmann::json _handleJobError(std::string const& func); +}; + +} // namespace lsst::qserv::czar + +#endif // LSST_QSERV_CZAR_HTTPCZARWORKERMODULE_H diff --git a/src/czar/HttpSvc.cc b/src/czar/HttpSvc.cc index cddaf17b3f..b67330e27d 100644 --- a/src/czar/HttpSvc.cc +++ b/src/czar/HttpSvc.cc @@ -28,6 +28,7 @@ // Qserv headers #include "cconfig/CzarConfig.h" #include "czar/HttpMonitorModule.h" +#include "czar/HttpCzarWorkerModule.h" #include "http/MetaModule.h" #include "qhttp/Server.h" @@ -90,6 +91,16 @@ uint16_t HttpSvc::start() { [self](shared_ptr const& req, shared_ptr const& resp) { HttpMonitorModule::process(::serviceName, req, resp, "STATUS"); }}}); + _httpServerPtr->addHandlers( + {{"POST", "/queryjob-error", + [self](shared_ptr const& req, shared_ptr const& resp) { + HttpCzarWorkerModule::process(::serviceName, req, resp, "QUERYJOB-ERROR"); + }}}); + _httpServerPtr->addHandlers( + {{"POST", "/queryjob-ready", + [self](shared_ptr const& req, shared_ptr const& resp) { + HttpCzarWorkerModule::process(::serviceName, req, resp, "QUERYJOB-READY"); + }}}); _httpServerPtr->start(); // Initialize the I/O context and start the service threads. At this point diff --git a/src/czar/MessageTable.cc b/src/czar/MessageTable.cc index 088aac835b..23020dc214 100644 --- a/src/czar/MessageTable.cc +++ b/src/czar/MessageTable.cc @@ -35,7 +35,7 @@ #include "ccontrol/ConfigMap.h" #include "ccontrol/UserQuery.h" #include "czar/CzarErrors.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" @@ -122,7 +122,7 @@ void MessageTable::_saveQueryMessages(ccontrol::UserQuery::Ptr const& userQuery) // Collect information about the query and put it in the message table. int msgCount = msgStore->messageCount(); for (int i = 0; i != msgCount; ++i) { - const qdisp::QueryMessage& qm = msgStore->getMessage(i); + const qmeta::QueryMessage& qm = msgStore->getMessage(i); std::string src = qm.msgSource; if (src == "COMPLETE") { ++completeCount; diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc new file mode 100644 index 0000000000..aad9fdfd31 --- /dev/null +++ b/src/czar/testCzar.cc @@ -0,0 +1,205 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include + +// Third-party headers +#include "boost/asio.hpp" +#include "nlohmann/json.hpp" + +// Boost unit test header +#define BOOST_TEST_MODULE Czar_1 +#include + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "qmeta/QMeta.h" +#include "czar/CzarChunkMap.h" + +namespace test = boost::test_tools; +using namespace lsst::qserv; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.testCzar"); +} + +using namespace std; + +BOOST_AUTO_TEST_SUITE(Suite) + +void insertIntoQChunkMap(qmeta::QMetaChunkMap& qChunkMap, string const& workerId, string const& dbName, + string const& tableName, unsigned int chunkNum, size_t sz) { + qChunkMap.workers[workerId][dbName][tableName].push_back(qmeta::QMetaChunkMap::ChunkInfo{chunkNum, sz}); +} + +qmeta::QMetaChunkMap convertJsonToChunkMap(nlohmann::json const& jsChunks) { + qmeta::QMetaChunkMap qChunkMap; + for (auto const& [workerId, dbs] : jsChunks.items()) { + for (auto const& [dbName, tables] : dbs.items()) { + for (auto const& [tableName, chunks] : tables.items()) { + for (auto const& [index, chunkNumNSz] : chunks.items()) { + try { + int64_t chunkNum = chunkNumNSz.at(0); + int64_t sz = chunkNumNSz.at(1); + LOGS(_log, LOG_LVL_DEBUG, + "workerdId=" << workerId << " db=" << dbName << " table=" << tableName + << " chunk=" << chunkNum << " sz=" << sz); + insertIntoQChunkMap(qChunkMap, workerId, dbName, tableName, chunkNum, sz); + } catch (invalid_argument const& exc) { + throw czar::ChunkMapException( + ERR_LOC, string(__func__) + " invalid_argument workerdId=" + workerId + + " db=" + dbName + " table=" + tableName + + " chunk=" + to_string(chunkNumNSz) + " " + exc.what()); + } catch (out_of_range const& exc) { + throw czar::ChunkMapException( + ERR_LOC, string(__func__) + " out_of_range workerdId=" + workerId + + " db=" + dbName + " table=" + tableName + + " chunk=" + to_string(chunkNumNSz) + " " + exc.what()); + } + } + } + } + } + return qChunkMap; +} + +BOOST_AUTO_TEST_CASE(CzarChunkMap) { + // Each chunk only occurs on one worker + string test1 = R"( + { + "ce1c1b79-e6fb-11ee-a46b-0242c0a80308": + {"qcase01": + {"Object":[[1234567890,0],[6630,1460],[6800,6068],[6968,1000],[6971,2716],[7140,4556],[7310,2144],[7648,1568]], + "Source":[[1234567890,0],[6630,37084],[6800,163888],[6968,33044],[6971,67016],[7140,145300],[7310,83872],[7648,30096]] + }, + "qcase02": + {"Object":[[1234567890,0],[7310,0]], + "Source":[[1234567890,0],[7310,0]] + }, + "qcase03": + {"RefDeepSrcMatch":[[1234567890,0],[7165,76356]], + "RefObject":[[1234567890,0],[7165,119616]], + "RunDeepForcedSource":[[1234567890,0],[7165,130617531]], + "RunDeepSource":[[1234567890,0],[7165,578396]] + } + }, + "ddc3f1b9-e6fb-11ee-a46b-0242c0a80304": + {"qcase01": + {"Object":[[1234567890,0],[6631,1612],[6801,4752],[6970,5780],[7138,3212],[7308,2144],[7478,4608]], + "Source":[[1234567890,0],[6631,45724],[6801,123940],[6970,151660],[7138,97252],[7308,56784],[7478,99304]] + }, + "qcase02": + {"Object":[[1234567890,0],[7480,1055000]], + "Source":[[1234567890,0],[7480,2259419]] + }, + "qcase03": + {"RefDeepSrcMatch":[[1234567890,0],[6995,7728]], + "RefObject":[[1234567890,0],[6995,10920]], + "RunDeepForcedSource":[[1234567890,0],[6995,11708834]], + "RunDeepSource":[[1234567890,0],[6995,58604]] + } + } + } + )"; + + /// 3 workers, each containing all chunks. + string test2 = R"( + { + "ce1c1b79-e6fb-11ee-a46b-0242c0a80308": + {"qcase01": + {"Object":[[1234567890,0],[6631,1612],[6801,4752],[6970,5780],[7138,3212],[7308,2144],[7478,4608], + [6630,1460],[6800,6068],[6968,1000],[6971,2716],[7140,4556],[7310,2144],[7648,1568]], + "Source":[[1234567890,0],[6631,45724],[6801,123940],[6970,151660],[7138,97252],[7308,56784],[7478,99304], + [6630,37084],[6800,163888],[6968,33044],[6971,67016],[7140,145300],[7310,83872],[7648,30096]] + }, + "qcase02": + {"Object":[[1234567890,0],[7480,1055000],[7310,0]], + "Source":[[1234567890,0],[7480,2259419],[7310,0]] + }, + "qcase03": + {"RefDeepSrcMatch":[[1234567890,0],[6995,7728],[7165,76356]], + "RefObject":[[1234567890,0],[6995,10920],[7165,119616]], + "RunDeepForcedSource":[[1234567890,0],[6995,11708834],[7165,130617531]], + "RunDeepSource":[[1234567890,0],[6995,58604],[7165,578396]] + } + }, + "brnd1b79-e6fb-11ee-a46b-0242c0a80308": + {"qcase01": + {"Object":[[1234567890,0],[6631,1612],[6801,4752],[6970,5780],[7138,3212],[7308,2144],[7478,4608], + [6630,1460],[6800,6068],[6968,1000],[6971,2716],[7140,4556],[7310,2144],[7648,1568]], + "Source":[[1234567890,0],[6631,45724],[6801,123940],[6970,151660],[7138,97252],[7308,56784],[7478,99304], + [6630,37084],[6800,163888],[6968,33044],[6971,67016],[7140,145300],[7310,83872],[7648,30096]] + }, + "qcase02": + {"Object":[[1234567890,0],[7480,1055000],[7310,0]], + "Source":[[1234567890,0],[7480,2259419],[7310,0]] + }, + "qcase03": + {"RefDeepSrcMatch":[[1234567890,0],[6995,7728],[7165,76356]], + "RefObject":[[1234567890,0],[6995,10920],[7165,119616]], + "RunDeepForcedSource":[[1234567890,0],[6995,11708834],[7165,130617531]], + "RunDeepSource":[[1234567890,0],[6995,58604],[7165,578396]] + } + }, + "ddc3f1b9-e6fb-11ee-a46b-0242c0a80304": + {"qcase01": + {"Object":[[1234567890,0],[6631,1612],[6801,4752],[6970,5780],[7138,3212],[7308,2144],[7478,4608], + [6630,1460],[6800,6068],[6968,1000],[6971,2716],[7140,4556],[7310,2144],[7648,1568]], + "Source":[[1234567890,0],[6631,45724],[6801,123940],[6970,151660],[7138,97252],[7308,56784],[7478,99304], + [6630,37084],[6800,163888],[6968,33044],[6971,67016],[7140,145300],[7310,83872],[7648,30096]] + }, + "qcase02": + {"Object":[[1234567890,0],[7480,1055000],[7310,0]], + "Source":[[1234567890,0],[7480,2259419],[7310,0]] + }, + "qcase03": + {"RefDeepSrcMatch":[[1234567890,0],[6995,7728],[7165,76356]], + "RefObject":[[1234567890,0],[6995,10920],[7165,119616]], + "RunDeepForcedSource":[[1234567890,0],[6995,11708834],[7165,130617531]], + "RunDeepSource":[[1234567890,0],[6995,58604],[7165,578396]] + } + } + } + )"; + + auto dbToFamily = make_shared(); + czar::CzarFamilyMap czFamMap(dbToFamily); + + auto jsTest1 = nlohmann::json::parse(test1); + qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); + auto familyMap = czFamMap.makeNewMaps(qChunkMap1); + czar::CzarFamilyMap::verify(familyMap); // Throws on failure. + LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 1 passed"); + + auto jsTest2 = nlohmann::json::parse(test2); + qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); + auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2); + czar::CzarFamilyMap::verify(familyMap2); // Throws on failure. + LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 2 passed"); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/global/clock_defs.h b/src/global/clock_defs.h index d582b416f8..9db4dadbc8 100644 --- a/src/global/clock_defs.h +++ b/src/global/clock_defs.h @@ -23,6 +23,7 @@ #define LSST_QSERV_GLOBAL_CLOCKDEFS_H // System headers +#include #include #include #include diff --git a/src/global/intTypes.h b/src/global/intTypes.h index 4182544f7f..c3a6f7fb07 100644 --- a/src/global/intTypes.h +++ b/src/global/intTypes.h @@ -37,6 +37,8 @@ typedef std::vector Int32Vector; /// Typedef for Query ID in query metadata. typedef std::uint64_t QueryId; +typedef std::int64_t JobId; +typedef JobId UberJobId; // These must be the same type. /// Class to provide a consistent format for QueryIds in the log file class QueryIdHelper { @@ -45,15 +47,15 @@ class QueryIdHelper { /// @parameter qid - query id number. /// @parameter invalid - true, qid is not a valid user query id. static std::string makeIdStr(QueryId qid, bool invalid = false) { - if (invalid) return "QI=?:"; - return "QI=" + std::to_string(qid) + ":"; + if (invalid) return "QID=?:"; + return "QID=" + std::to_string(qid) + ":"; } /// Returns a standardized user query id string with jobId. /// @parameter qid - query id number. /// @parameter jobId - the job id number. /// @parameter invalid - true, qid is not a valid user query id. - static std::string makeIdStr(QueryId qid, int jobId, bool invalid = false) { + static std::string makeIdStr(QueryId qid, JobId jobId, bool invalid = false) { if (invalid) return makeIdStr(qid, true) + "?;"; return makeIdStr(qid) + std::to_string(jobId) + ";"; } diff --git a/src/http/Module.h b/src/http/Module.h index 8a89a4e8f2..b97f4c47e3 100644 --- a/src/http/Module.h +++ b/src/http/Module.h @@ -85,6 +85,16 @@ class Module : public BaseModule { */ virtual nlohmann::json executeImpl(std::string const& subModuleName) = 0; + /** + * Send a response back to a requester of a service. + * @param content The content to be sent back. + * @param contentType The type of the content to be sent back. + */ + virtual void sendResponse(std::string const& content, std::string const& contentType) = 0; + + std::string authKey() const { return _authKey; } // &&&uj + + private: /** * Pull the raw request body and translate it into a JSON object. diff --git a/src/http/RequestBodyJSON.h b/src/http/RequestBodyJSON.h index 6bf6a8068f..b88858a1dd 100644 --- a/src/http/RequestBodyJSON.h +++ b/src/http/RequestBodyJSON.h @@ -42,6 +42,17 @@ class RequestBodyJSON { /// parsed body of the request nlohmann::json objJson = nlohmann::json::object(); + RequestBody() = default; + RequestBody(RequestBody const&) = default; + RequestBody& operator=(RequestBody const&) = default; + + ~RequestBody() = default; + + /// Make a new RequestBody based on `js` + /// TODO:UJ This would be much more efficient if this class had objJson defined as + /// a const reference or pointer to const, but implementation is likely ugly. + RequestBody(nlohmann::json const& js) : objJson(js) {} + /** * Check if the specified parameter is present in the input JSON object. * @param obj JSON object to be inspected. @@ -73,8 +84,15 @@ class RequestBodyJSON { throw std::invalid_argument("RequestBodyJSON::" + std::string(__func__) + "[static] parameter 'obj' is not a valid JSON object"); } +<<<<<<< HEAD:src/http/RequestBodyJSON.h if (obj.find(name) != obj.end()) return obj[name]; throw std::invalid_argument("RequestBodyJSON::" + std::string(__func__) + +======= + if (auto const iter = obj.find(name); iter != obj.end()) { + return *iter; + } + throw std::invalid_argument("RequestBody::" + std::string(__func__) + +>>>>>>> 4c670c16d (Czar and workers can send http messages to each other.):src/http/RequestBody.h "[static] required parameter " + name + " is missing in the request body"); } diff --git a/src/proto/ScanTableInfo.h b/src/proto/ScanTableInfo.h index 634953e656..f2dacec61a 100644 --- a/src/proto/ScanTableInfo.h +++ b/src/proto/ScanTableInfo.h @@ -38,6 +38,7 @@ namespace lsst::qserv::proto { struct ScanTableInfo { using ListOf = std::vector; + ScanTableInfo() = default; ScanTableInfo(std::string const& db_, std::string const& table_) : db(db_), table(table_) {} ScanTableInfo(std::string const& db_, std::string const& table_, bool lockInMemory_, int scanRating_) : db{db_}, table{table_}, lockInMemory{lockInMemory_}, scanRating{scanRating_} {} @@ -47,6 +48,8 @@ struct ScanTableInfo { lockInMemory{scanTbl.lockinmemory()}, scanRating{scanTbl.scanrating()} {} + ScanTableInfo(ScanTableInfo const&) = default; + /// Copy contents of this object into a TaskMsg_ScanTable object. void copyToScanTable(TaskMsg_ScanTable* msgScanTbl) const { msgScanTbl->set_db(db); @@ -67,6 +70,9 @@ struct ScanInfo { /// Threshold priority values. Scan priorities are not limited to these values. enum Rating { FASTEST = 0, FAST = 10, MEDIUM = 20, SLOW = 30, SLOWEST = 100 }; + ScanInfo() = default; + ScanInfo(ScanInfo const&) = default; + void sortTablesSlowestFirst(); int compareTables(ScanInfo const& rhs); diff --git a/src/proto/worker.proto b/src/proto/worker.proto index 52c19e3929..46f3e69efd 100644 --- a/src/proto/worker.proto +++ b/src/proto/worker.proto @@ -29,6 +29,7 @@ option cc_enable_arenas = true; package lsst.qserv.proto; +// TODO:UJ delete when xrootd removed. ResonseSummary will need to be kept. // Query message sent to worker // One of these Task objects should be sent. message TaskMsg { diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index aacde12776..74915ec984 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -5,12 +5,12 @@ target_sources(qdisp PRIVATE ChunkMeta.cc CzarStats.cc Executive.cc + JobBase.cc JobDescription.cc JobQuery.cc - JobStatus.cc - MessageStore.cc QdispPool.cc QueryRequest.cc + UberJob.cc XrdSsiMocks.cc ) diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 51b1da226d..0df6f246c7 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -58,21 +58,25 @@ // Qserv headers #include "cconfig/CzarConfig.h" +#include "ccontrol/MergingHandler.h" #include "ccontrol/msgCode.h" +#include "ccontrol/TmpTableName.h" +#include "ccontrol/UserQuerySelect.h" #include "global/LogContext.h" #include "global/ResourceUnit.h" #include "qdisp/CzarStats.h" #include "qdisp/JobQuery.h" -#include "qdisp/MessageStore.h" #include "qdisp/QueryRequest.h" #include "qdisp/ResponseHandler.h" #include "qdisp/XrdSsiMocks.h" #include "query/QueryContext.h" #include "qproc/QuerySession.h" #include "qmeta/Exceptions.h" +#include "qmeta/MessageStore.h" #include "qmeta/QProgress.h" #include "qmeta/QProgressHistory.h" #include "query/SelectStmt.h" +#include "rproc/InfileMerger.h" #include "util/AsyncTimer.h" #include "util/Bug.h" #include "util/EventThread.h" @@ -118,8 +122,13 @@ Executive::Executive(ExecutiveConfig const& c, shared_ptr const& m } Executive::~Executive() { + LOGS(_log, LOG_LVL_DEBUG, "Executive::~Executive() " << getIdStr()); qdisp::CzarStats::get()->deleteQuery(); qdisp::CzarStats::get()->deleteJobs(_incompleteJobs.size()); + // Remove this executive from the map. + if (czar::Czar::getCzar()->getExecutiveFromMap(getId()) != nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) + " pointer in map should be invalid QID=" << getId()); + } // Real XrdSsiService objects are unowned, but mocks are allocated in _setup. delete dynamic_cast(_xrdSsiService); if (_asyncTimer != nullptr) { @@ -134,7 +143,7 @@ Executive::~Executive() { } } -Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptr const& ms, +Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptr const& ms, SharedResources::Ptr const& sharedResources, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, @@ -154,17 +163,20 @@ Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptrczarStatsUpdateIvalSec(); if (czarStatsUpdateIvalSec > 0) { + // AsyncTimer has a 'self' keep alive in AsyncTimer::start() that keeps it safe when + // this Executive is deleted. ptr->_asyncTimer = util::AsyncTimer::create( asioIoService, std::chrono::milliseconds(czarStatsUpdateIvalSec * 1000), [self = std::weak_ptr(ptr)](auto expirationIvalMs) -> bool { auto ptr = self.lock(); - LOGS(_log, LOG_LVL_DEBUG, - "Executive::" << __func__ << " expirationIvalMs: " << expirationIvalMs.count() - << " ms"); + string const msg = string("Executive::") + __func__ + + " expirationIvalMs: " + to_string(expirationIvalMs.count()) + " ms"; if (ptr != nullptr) { ptr->_updateStats(); + LOGS(_log, LOG_LVL_DEBUG, msg + " " + ptr->getIdStr()); return true; } + LOGS(_log, LOG_LVL_DEBUG, msg); return false; }); ptr->_asyncTimer->start(); @@ -183,6 +195,10 @@ void Executive::_updateStats() const { } void Executive::setQueryId(QueryId id) { + if (_queryIdSet.exchange(true) == true) { + throw util::Bug(ERR_LOC, "Executive::setQueryId called more than once _id=" + to_string(_id) + + " id=" + to_string(id)); + } _id = id; _idStr = QueryIdHelper::makeIdStr(_id); if (_queryProgressHistory != nullptr) { @@ -194,18 +210,27 @@ void Executive::setQueryId(QueryId id) { } } +UberJob::Ptr Executive::findUberJob(UberJobId ujId) { + lock_guard lgMap(_uberJobsMapMtx); + auto iter = _uberJobsMap.find(ujId); + if (iter == _uberJobsMap.end()) { + return nullptr; + } + return iter->second; +} + /// Add a new job to executive queue, if not already in. Not thread-safe. /// JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { JobQuery::Ptr jobQuery; { // Create the JobQuery and put it in the map. - JobStatus::Ptr jobStatus = make_shared(); + auto jobStatus = make_shared(); Ptr thisPtr = shared_from_this(); MarkCompleteFunc::Ptr mcf = make_shared(thisPtr, jobDesc->id()); jobQuery = JobQuery::create(thisPtr, jobDesc, jobStatus, mcf, _id); - QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getIdInt()); + QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getJobId()); { lock_guard lock(_cancelled.getMutex()); @@ -220,10 +245,12 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { return jobQuery; } - if (!_track(jobQuery->getIdInt(), jobQuery)) { + if (!_track(jobQuery->getJobId(), jobQuery)) { LOGS(_log, LOG_LVL_ERROR, "Executive ignoring duplicate track add"); return jobQuery; } + + _addToChunkJobMap(jobQuery); } if (_empty.exchange(false)) { @@ -232,14 +259,16 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { ++_requestCount; } - QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getIdInt()); + QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getJobId()); - LOGS(_log, LOG_LVL_DEBUG, "Executive::add with path=" << jobDesc->resource().path()); + return jobQuery; +} + +void Executive::runJobQuery(JobQuery::Ptr const& jobQuery) { bool started = jobQuery->runJob(); if (!started && isLimitRowComplete()) { - markCompleted(jobQuery->getIdInt(), false); + markCompleted(jobQuery->getJobId(), false); } - return jobQuery; } void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { @@ -251,6 +280,32 @@ void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { } } +void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 3); + } else { + _qdispPool->queCmd(cmd, 4); + } +} + +void Executive::runUberJob(std::shared_ptr const& uberJob) { + /// TODO:UJ delete useqdisppool, only set to false if problems during testing + bool const useqdisppool = true; + if (useqdisppool) { + auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; + + auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); + _jobStartCmdList.push_back(cmd); + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 0); + } else { + _qdispPool->queCmd(cmd, 1); + } + } else { + uberJob->runUberJob(); + } +} + void Executive::waitForAllJobsToStart() { LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart"); // Wait for each command to start. @@ -266,12 +321,11 @@ void Executive::waitForAllJobsToStart() { // If the executive has not been cancelled, then we simply start the query. // @return true if query was actually started (i.e. we were not cancelled) -// +// // TODO:UJ delete this function bool Executive::startQuery(shared_ptr const& jobQuery) { lock_guard lock(_cancelled.getMutex()); - // If we have been cancelled, then return false. - // + // If this has been cancelled, then return false. if (_cancelled) return false; // Construct a temporary resource object to pass to ProcessRequest(). @@ -294,11 +348,67 @@ bool Executive::startQuery(shared_ptr const& jobQuery) { return true; } +Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { + lock_guard lck(_chunkToJobMapMtx); + + ChunkIdJobMapType unassignedMap; + for (auto const& [key, jobPtr] : _chunkToJobMap) { + if (!jobPtr->isInUberJob()) { + unassignedMap[key] = jobPtr; + } + } + return unassignedMap; +} + +void Executive::addUberJobs(std::vector> const& uJobsToAdd) { + lock_guard lck(_uberJobsMapMtx); + for (auto const& uJob : uJobsToAdd) { + UberJobId ujId = uJob->getJobId(); + _uberJobsMap[ujId] = uJob; + } +} + +string Executive::dumpUberJobCounts() const { + stringstream os; + os << "exec=" << getIdStr(); + int totalJobs = 0; + { + lock_guard ujmLck(_uberJobsMapMtx); + for (auto const& [ujKey, ujPtr] : _uberJobsMap) { + int jobCount = ujPtr->getJobCount(); + totalJobs += jobCount; + os << "{" << ujKey << ":" << ujPtr->getIdStr() << " jobCount=" << jobCount << "}"; + } + } + { + lock_guard jmLck(_jobMapMtx); + os << " ujTotalJobs=" << totalJobs << " execJobs=" << _jobMap.size(); + } + return os.str(); +} + +void Executive::assignJobsToUberJobs() { + auto uqs = _userQuerySelect.lock(); + if (uqs != nullptr) { + uqs->buildAndSendUberJobs(); + } +} + +void Executive::addMultiError(int errorCode, std::string const& errorMsg, int errorState) { + util::Error err(errorCode, errorMsg, errorState); + { + lock_guard lock(_errorsMutex); + _multiError.push_back(err); + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) + " multiError:" << _multiError.size() << ":" << _multiError); + } +} + /// Add a JobQuery to this Executive. /// Return true if it was successfully added to the map. /// bool Executive::_addJobToMap(JobQuery::Ptr const& job) { - auto entry = pair(job->getIdInt(), job); + auto entry = pair(job->getJobId(), job); lock_guard lockJobMap(_jobMapMtx); bool res = _jobMap.insert(entry).second; _totalJobs = _jobMap.size(); @@ -312,9 +422,10 @@ bool Executive::join() { // Okay to merge. probably not the Executive's responsibility struct successF { static bool func(Executive::JobMap::value_type const& entry) { - JobStatus::Info const& esI = entry.second->getStatus()->getInfo(); + qmeta::JobStatus::Info const& esI = entry.second->getStatus()->getInfo(); LOGS(_log, LOG_LVL_TRACE, "entry state:" << (void*)entry.second.get() << " " << esI); - return (esI.state == JobStatus::RESPONSE_DONE) || (esI.state == JobStatus::COMPLETE); + return (esI.state == qmeta::JobStatus::RESPONSE_DONE) || + (esI.state == qmeta::JobStatus::COMPLETE); } }; @@ -341,7 +452,7 @@ bool Executive::join() { return _empty || isLimitRowComplete(); } -void Executive::markCompleted(int jobId, bool success) { +void Executive::markCompleted(JobId jobId, bool success) { ResponseHandler::Error err; string idStr = QueryIdHelper::makeIdStr(_id, jobId); LOGS(_log, LOG_LVL_DEBUG, "Executive::markCompleted " << success); @@ -372,13 +483,10 @@ void Executive::markCompleted(int jobId, bool success) { lock_guard lockJobMap(_jobMapMtx); auto job = _jobMap[jobId]; string id = job->getIdStr() + "<>" + idStr; - auto jState = job->getStatus()->getInfo().state; + // Don't overwrite existing error states. - if (jState != JobStatus::CANCEL && jState != JobStatus::RESPONSE_ERROR && - jState != JobStatus::RESULT_ERROR && jState != JobStatus::MERGE_ERROR) { - job->getStatus()->updateInfo(id, JobStatus::RESULT_ERROR, "EXECFAIL", err.getCode(), - err.getMsg()); - } + job->getStatus()->updateInfoNoErrorOverwrite(id, qmeta::JobStatus::RESULT_ERROR, "EXECFAIL", + err.getCode(), err.getMsg()); } { lock_guard lock(_errorsMutex); @@ -399,11 +507,11 @@ void Executive::markCompleted(int jobId, bool success) { void Executive::squash() { bool alreadyCancelled = _cancelled.exchange(true); if (alreadyCancelled) { - LOGS(_log, LOG_LVL_DEBUG, "Executive::squash() already cancelled! refusing."); + LOGS(_log, LOG_LVL_DEBUG, "Executive::squash() already cancelled! refusing. qid=" << getId()); return; } - LOGS(_log, LOG_LVL_INFO, "Executive::squash Trying to cancel all queries..."); + LOGS(_log, LOG_LVL_INFO, "Executive::squash Trying to cancel all queries... qid=" << getId()); deque jobsToCancel; { lock_guard lockJobMap(_jobMapMtx); @@ -415,6 +523,13 @@ void Executive::squash() { for (auto const& job : jobsToCancel) { job->cancel(); } + + // TODO:UJ - Send a message to all workers saying this czarId + queryId is cancelled. + // The workers will just mark all associated tasks as cancelled, and that should be it. + // Any message to this czar about this query should result in an error sent back to + // the worker as soon it can't locate an executive or the executive says cancelled. + bool const deleteResults = true; + sendWorkerCancelMsg(deleteResults); LOGS(_log, LOG_LVL_DEBUG, "Executive::squash done"); } @@ -432,7 +547,8 @@ void Executive::_squashSuperfluous() { JobQuery::Ptr jq = jobEntry.second; // It's important that none of the cancelled queries // try to remove their rows from the result. - if (jq->getStatus()->getInfo().state != JobStatus::COMPLETE) { + if (jq->getStatus()->getInfo().state != qmeta::JobStatus::COMPLETE && + jq->getStatus()->getInfo().state != qmeta::JobStatus::CANCEL) { jobsToCancel.push_back(jobEntry.second); } } @@ -441,9 +557,22 @@ void Executive::_squashSuperfluous() { for (auto const& job : jobsToCancel) { job->cancel(true); } + + bool const keepResults = false; + sendWorkerCancelMsg(keepResults); LOGS(_log, LOG_LVL_DEBUG, "Executive::squashSuperfluous done"); } +void Executive::sendWorkerCancelMsg(bool deleteResults) { + // TODO:UJ need to send a message to the worker that the query is cancelled and all result files + // should be delete + LOGS(_log, LOG_LVL_ERROR, + "TODO:UJ NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId " + "+ " + "queryId. " + << deleteResults); +} + int Executive::getNumInflight() const { unique_lock lock(_incompleteJobsMutex); return _incompleteJobs.size(); @@ -640,6 +769,16 @@ void Executive::_waitAllUntilEmpty() { } } +void Executive::_addToChunkJobMap(JobQuery::Ptr const& job) { + int chunkId = job->getDescription()->resource().chunk(); + auto entry = pair(chunkId, job); + lock_guard lck(_chunkToJobMapMtx); + bool inserted = _chunkToJobMap.insert(entry).second; + if (!inserted) { + throw util::Bug(ERR_LOC, "map insert FAILED ChunkId=" + to_string(chunkId) + " already existed"); + } +} + void Executive::_setupLimit() { // Figure out the limit situation. auto qSession = _querySession.lock(); @@ -675,12 +814,12 @@ void Executive::checkLimitRowComplete() { } ostream& operator<<(ostream& os, Executive::JobMap::value_type const& v) { - JobStatus::Ptr status = v.second->getStatus(); + auto const& status = v.second->getStatus(); os << v.first << ": " << *status; return os; } -/// precondition: _requestersMutex is held by current thread. +/// precondition: _incompleteJobsMutex is held by current thread. void Executive::_printState(ostream& os) { for (auto const& entry : _incompleteJobs) { JobQuery::Ptr job = entry.second; diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index ce4dc3e521..cc68b85a89 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -30,7 +30,6 @@ #include #include #include -#include #include // Third-party headers @@ -41,16 +40,20 @@ #include "global/ResourceUnit.h" #include "global/stringTypes.h" #include "qdisp/JobDescription.h" -#include "qdisp/JobStatus.h" #include "qdisp/ResponseHandler.h" #include "qdisp/SharedResources.h" #include "qdisp/QdispPool.h" +#include "qdisp/UberJob.h" +#include "qmeta/JobStatus.h" #include "util/EventThread.h" #include "util/InstanceCount.h" #include "util/MultiError.h" #include "util/threadSafe.h" #include "util/ThreadPool.h" +// TODO:UJ replace with better enable/disable feature, or just use only UberJobs +#define uberJobsEnabled 1 + // Forward declarations class XrdSsiService; @@ -59,16 +62,25 @@ class QProgress; class QProgressHistory; } // namespace lsst::qserv::qmeta +namespace ccontrol { +class UserQuerySelect; +} + +namespace qmeta { +class MessageStore; +} // namespace qmeta + namespace lsst::qserv::qproc { class QuerySession; } // namespace lsst::qserv::qproc namespace lsst::qserv::qdisp { class JobQuery; -class MessageStore; -} // namespace lsst::qserv::qdisp +class UberJob; +} // namespace qdisp -namespace lsst::qserv::util { + +namespace util { class AsyncTimer; } // namespace lsst::qserv::util @@ -86,17 +98,18 @@ struct ExecutiveConfig { static std::string getMockStr() { return "Mock"; } }; -/// class Executive manages the execution of jobs for a UserQuery, while -/// maintaining minimal information about the jobs themselves. +/// class Executive manages the execution of jobs for a UserQuery. class Executive : public std::enable_shared_from_this { public: typedef std::shared_ptr Ptr; typedef std::unordered_map> JobMap; + typedef int ChunkIdType; + typedef std::map> ChunkIdJobMapType; /// Construct an Executive. /// If c->serviceUrl == ExecutiveConfig::getMockStr(), then use XrdSsiServiceMock /// instead of a real XrdSsiService - static Executive::Ptr create(ExecutiveConfig const& c, std::shared_ptr const& ms, + static Executive::Ptr create(ExecutiveConfig const& c, std::shared_ptr const& ms, SharedResources::Ptr const& sharedResources, std::shared_ptr const& queryProgress, std::shared_ptr const& queryProgressHistory, @@ -105,12 +118,33 @@ class Executive : public std::enable_shared_from_this { ~Executive(); + std::string cName(const char* funcName = "") { return std::string("Executive::") + funcName; } + + /// Set the UserQuerySelect object for this query so this Executive can ask it to make new + /// UberJobs in the future, if needed. + void setUserQuerySelect(std::shared_ptr const& uqs) { _userQuerySelect = uqs; } + + /// Return a map that only contains Jobs not assigned to an UberJob. + ChunkIdJobMapType unassignedChunksInQuery(); + + /// Find the UberJob with `ujId`. + std::shared_ptr findUberJob(UberJobId ujId); + /// Add an item with a reference number std::shared_ptr add(JobDescription::Ptr const& s); + /// TODO:UJ - to be deleted + void runJobQuery(std::shared_ptr const& jobQuery); + + // Queue `uberJob` to be run using the QDispPool. + void runUberJob(std::shared_ptr const& uberJob); + /// Queue a job to be sent to a worker so it can be started. void queueJobStart(PriorityCommand::Ptr const& cmd); + /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. + void queueFileCollect(PriorityCommand::Ptr const& cmd); + /// Waits for all jobs on _jobStartCmdList to start. This should not be called /// before ALL jobs have been added to the pool. void waitForAllJobsToStart(); @@ -120,14 +154,18 @@ class Executive : public std::enable_shared_from_this { bool join(); /// Notify the executive that an item has completed - void markCompleted(int refNum, bool success); + void markCompleted(JobId refNum, bool success); /// Squash all the jobs. void squash(); bool getEmpty() { return _empty; } + /// These values cannot be set until information has been collected from + /// QMeta, which isn't called until some basic checks on the user query + /// have passed. void setQueryId(QueryId id); + QueryId getId() const { return _id; } std::string const& getIdStr() const { return _idStr; } @@ -146,7 +184,7 @@ class Executive : public std::enable_shared_from_this { std::shared_ptr getQdispPool() { return _qdispPool; } - bool startQuery(std::shared_ptr const& jobQuery); + bool startQuery(std::shared_ptr const& jobQuery); // TODO:UJ delete /// Add 'rowCount' to the total number of rows in the result table. void addResultRows(int64_t rowCount); @@ -168,8 +206,43 @@ class Executive : public std::enable_shared_from_this { /// @see python module lsst.qserv.czar.proxy.unlock() void updateProxyMessages(); + /// Add UbjerJobs to this user query. + void addUberJobs(std::vector> const& jobsToAdd); + + /// Call UserQuerySelect::buildAndSendUberJobs make new UberJobs for + /// unassigned jobs. + void assignJobsToUberJobs(); + + int getTotalJobs() { return _totalJobs; } + + /// Set `_failedUberJob` to `val`; Setting this to true is a flag + /// that indicates to the Czar::_monitor that this Executive + /// probably has unassigned jobs that need to be placed in + /// new UberJobs. This `val` should only be set false by + /// Czar::_monitor(). + void setFlagFailedUberJob(bool val) { _failedUberJob = val; } + + /// Add an error code and message that may be displayed to the user. + void addMultiError(int errorCode, std::string const& errorMsg, int errState); + + std::string dumpUberJobCounts() const; + + // The below value should probably be based on the user query, with longer sleeps for slower queries. + int getAttemptSleepSeconds() const { return 15; } // As above or until added to config file. + int getMaxAttempts() const { return 5; } // Should be set by config + + /// Calling this indicates the executive is ready to create and execute UberJobs. + void setReadyToExecute() { _readyToExecute = true; } + + /// Returns true if the executive is ready to create and execute UberJobs. + bool isReadyToExecute() { return _readyToExecute; } + + /// Send a message to all workers to cancel this query. + /// @param deleteResults - If true, delete all result files for this query on the workers. + void sendWorkerCancelMsg(bool deleteResults); + private: - Executive(ExecutiveConfig const& c, std::shared_ptr const& ms, + Executive(ExecutiveConfig const& c, std::shared_ptr const& ms, SharedResources::Ptr const& sharedResources, std::shared_ptr const& queryProgress, std::shared_ptr const& queryProgressHistory, @@ -201,7 +274,7 @@ class Executive : public std::enable_shared_from_this { ExecutiveConfig _config; ///< Personal copy of config std::atomic _empty{true}; - std::shared_ptr _messageStore; ///< MessageStore for logging + std::shared_ptr _messageStore; ///< MessageStore for logging /// RPC interface, static to avoid getting every time a user query starts and separate /// from _xrdSsiService to avoid conflicts with XrdSsiServiceMock. @@ -228,11 +301,13 @@ class Executive : public std::enable_shared_from_this { mutable std::mutex _errorsMutex; std::condition_variable _allJobsComplete; + // TODO:UJ see what it takes to make this a normal mutex, before + // xrootd resulted in things being called in difficult to predict + // ways. That shouldn't be an issue any more. mutable std::recursive_mutex _jobMapMtx; - QueryId _id{0}; ///< Unique identifier for this query. + QueryId _id = 0; ///< Unique identifier for this query. std::string _idStr{QueryIdHelper::makeIdStr(0, true)}; - // util::InstanceCount _instC{"Executive"}; std::shared_ptr _queryProgress; ///< Query progress, used to update QMeta. std::shared_ptr @@ -245,6 +320,18 @@ class Executive : public std::enable_shared_from_this { bool _scanInteractive = false; ///< true for interactive scans. + // Add a job to the _chunkToJobMap + // TODO:UJ This may need review as large changes were made to this part of the code. + // code is no longer destructive to _chunkToJobMap + void _addToChunkJobMap(std::shared_ptr const& job); + std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap + ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId + + /// Map of all UberJobs. Failed UberJobs remain in the map as new ones are created + /// to handle failed UberJobs. + std::map> _uberJobsMap; + mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. + /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. std::atomic _limitRowComplete{false}; @@ -259,13 +346,26 @@ class Executive : public std::enable_shared_from_this { /// Number of time data has been ignored for for this user query. std::atomic _dataIgnoredCount{0}; + + std::atomic _queryIdSet{false}; ///< Set to true when _id is set. + + /// Weak pointer to the UserQuerySelect object for this query. + std::weak_ptr _userQuerySelect; + + /// If this is true, there are probably jobs that need to + /// be reassigned to new UberJobs. + std::atomic _failedUberJob{false}; + + /// Flag that is set to true when ready to create and run UberJobs. + std::atomic _readyToExecute{false}; }; +/// TODO:UJ delete - MarkCompleteFunc is not needed with uberjobs. class MarkCompleteFunc { public: typedef std::shared_ptr Ptr; - MarkCompleteFunc(Executive::Ptr const& e, int jobId) : _executive(e), _jobId(jobId) {} + MarkCompleteFunc(Executive::Ptr const& e, JobId jobId) : _executive(e), _jobId(jobId) {} virtual ~MarkCompleteFunc() {} virtual void operator()(bool success) { @@ -277,7 +377,7 @@ class MarkCompleteFunc { private: std::weak_ptr _executive; - int _jobId; + JobId _jobId; }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobBase.cc b/src/qdisp/JobBase.cc new file mode 100644 index 0000000000..a5ef5a8c8f --- /dev/null +++ b/src/qdisp/JobBase.cc @@ -0,0 +1,54 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "qdisp/JobBase.h" + +// System headers +#include + +// Qserv headers + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobBase"); +} + +namespace lsst { namespace qserv { namespace qdisp { + +std::ostream& JobBase::dumpOS(std::ostream& os) const { + os << "JobBase no data members"; + return os; +} + +std::string JobBase::dump() const { + std::ostringstream os; + dumpOS(os); + return os.str(); +} + +std::ostream& operator<<(std::ostream& os, JobBase const& jb) { return jb.dumpOS(os); } + +}}} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h new file mode 100644 index 0000000000..e5df5fc2ab --- /dev/null +++ b/src/qdisp/JobBase.h @@ -0,0 +1,78 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_QDISP_JOBBASE_H +#define LSST_QSERV_QDISP_JOBBASE_H + +// System headers +#include +#include + +// Qserv headers +#include "global/intTypes.h" + +namespace lsst::qserv::qmeta { +class JobStatus; +} + +// This header declarations +namespace lsst::qserv::qdisp { + +class Executive; +class QdispPool; +class ResponseHandler; +class QueryRequest; + +/// Base class for JobQuery and UberJob. +/// TODO:UJ This could use a lot of cleanup. Once UberJobs are fully in effect, there's no need +/// for this base class as it won't be possible to send a JobQuery to a worker without +/// putting it in an UberJob first. The UberJob is a wrapper that stores worker contact +/// info. +class JobBase : public std::enable_shared_from_this { +public: + using Ptr = std::shared_ptr; + + JobBase() = default; + JobBase(JobBase const&) = delete; + JobBase& operator=(JobBase const&) = delete; + virtual ~JobBase() = default; + + virtual QueryId getQueryId() const = 0; + virtual UberJobId getJobId() const = 0; + virtual std::string const& getIdStr() const = 0; + virtual std::shared_ptr getQdispPool() = 0; + virtual std::string const& getPayload() const = 0; ///< const& in return type is essential for xrootd + virtual std::shared_ptr getRespHandler() = 0; + virtual std::shared_ptr getStatus() = 0; + virtual bool getScanInteractive() const = 0; + virtual bool isQueryCancelled() = 0; + virtual void callMarkCompleteFunc(bool success) = 0; + virtual void setQueryRequest(std::shared_ptr const& qr) = 0; + virtual std::shared_ptr getExecutive() = 0; + + virtual std::ostream& dumpOS(std::ostream& os) const; + + std::string dump() const; + friend std::ostream& operator<<(std::ostream& os, JobBase const& jb); +}; + +} // namespace lsst::qserv::qdisp + +#endif // LSST_QSERV_QDISP_JOBBASE_H diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index dfa87e22ac..c8ade0f58e 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -35,6 +35,8 @@ // Qserv headers #include "proto/ProtoImporter.h" #include "proto/worker.pb.h" +#include "util/Bug.h" +#include "qdisp/Executive.h" #include "qdisp/ResponseHandler.h" #include "qproc/ChunkQuerySpec.h" #include "qproc/TaskMsgFactory.h" @@ -47,7 +49,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobDescription"); namespace lsst::qserv::qdisp { -JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, int jobId, ResourceUnit const& resource, +JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, shared_ptr const& respHandler, shared_ptr const& taskMsgFactory, shared_ptr const& chunkQuerySpec, @@ -73,6 +75,39 @@ bool JobDescription::incrAttemptCount() { return true; } +bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase) { + if (increase) { + ++_attemptCount; + } + if (_attemptCount >= MAX_JOB_ATTEMPTS) { + LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); + return false; + } + + if (exec != nullptr) { + int maxAttempts = exec->getMaxAttempts(); + LOGS(_log, LOG_LVL_INFO, "JoQDescription::" << __func__ << " attempts=" << _attemptCount); + if (_attemptCount > maxAttempts) { + LOGS(_log, LOG_LVL_ERROR, + "JoQDescription::" << __func__ << " attempts(" << _attemptCount << ") > maxAttempts(" + << maxAttempts << ") cancelling"); + exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, + "max attempts reached " + to_string(_attemptCount) + " " + _qIdStr, + util::ErrorCode::INTERNAL); + exec->squash(); + return false; + } + } + + // build the request + auto js = _taskMsgFactory->makeMsgJson(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, + _attemptCount, _czarId); + LOGS(_log, LOG_LVL_DEBUG, "JobDescription::" << __func__ << " js=" << (*js)); + _jsForWorker = js; + + return true; +} + void JobDescription::buildPayload() { ostringstream os; _taskMsgFactory->serializeMsg(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, _attemptCount, @@ -80,7 +115,7 @@ void JobDescription::buildPayload() { _payloads[_attemptCount] = os.str(); } -bool JobDescription::verifyPayload() const { +bool JobDescription::verifyPayload() const { // TODO:UJ delete proto::ProtoImporter pi; if (!_mock && !pi.messageAcceptable(_payloads.at(_attemptCount))) { LOGS(_log, LOG_LVL_DEBUG, _qIdStr << " Error serializing TaskMsg."); diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 6ecfc2e9a8..bc69edb25a 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -31,6 +31,9 @@ #include #include +// Third party headers +#include "nlohmann/json.hpp" + // Qserv headers #include "global/constants.h" #include "global/intTypes.h" @@ -41,15 +44,18 @@ namespace lsst::qserv { -namespace qproc { +namespace proto { +class TaskMsg; +} +namespace qproc { class ChunkQuerySpec; class TaskMsgFactory; - } // namespace qproc namespace qdisp { +class Executive; class ResponseHandler; /** Description of a job managed by the executive @@ -57,7 +63,7 @@ class ResponseHandler; class JobDescription { public: using Ptr = std::shared_ptr; - static JobDescription::Ptr create(qmeta::CzarId czarId, QueryId qId, int jobId, + static JobDescription::Ptr create(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, std::shared_ptr const& respHandler, std::shared_ptr const& taskMsgFactory, @@ -72,7 +78,7 @@ class JobDescription { JobDescription& operator=(JobDescription const&) = delete; void buildPayload(); ///< Must be run after construction to avoid problems with unit tests. - int id() const { return _jobId; } + JobId id() const { return _jobId; } ResourceUnit const& resource() const { return _resource; } std::string const& payload() { return _payloads[_attemptCount]; } std::shared_ptr respHandler() { return _respHandler; } @@ -85,17 +91,22 @@ class JobDescription { bool incrAttemptCount(); bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. + std::shared_ptr getJsForWorker() { return _jsForWorker; } + + void resetJsForWorker() { _jsForWorker.reset(); } // TODO:UJ may need mutex for _jsForWorker + friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); private: - JobDescription(qmeta::CzarId czarId, QueryId qId, int jobId, ResourceUnit const& resource, + JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, std::shared_ptr const& respHandler, std::shared_ptr const& taskMsgFactory, std::shared_ptr const& chunkQuerySpec, std::string const& chunkResultName, bool mock = false); + qmeta::CzarId _czarId; QueryId _queryId; - int _jobId; ///< Job's Id number. + JobId _jobId; ///< Job's Id number. std::string const _qIdStr; int _attemptCount{-1}; ///< Start at -1 so that first attempt will be 0, see incrAttemptCount(). ResourceUnit _resource; ///< path, e.g. /q/LSST/23125 @@ -112,6 +123,9 @@ class JobDescription { std::string _chunkResultName; bool _mock{false}; ///< True if this is a mock in a unit test. + + /// The information the worker needs to run this job. Reset once sent. + std::shared_ptr _jsForWorker; }; std::ostream& operator<<(std::ostream& os, JobDescription const& jd); diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 03894965f0..9149311968 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -40,45 +40,52 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobQuery"); } // anonymous namespace +using namespace std; + namespace lsst::qserv::qdisp { JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - JobStatus::Ptr const& jobStatus, std::shared_ptr const& markCompleteFunc, - QueryId qid) - : _executive(executive), + qmeta::JobStatus::Ptr const& jobStatus, + shared_ptr const& markCompleteFunc, QueryId qid) + : JobBase(), + _executive(executive), _jobDescription(jobDescription), _markCompleteFunc(markCompleteFunc), _jobStatus(jobStatus), _qid(qid), - _idStr(QueryIdHelper::makeIdStr(qid, getIdInt())) { + _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { _qdispPool = executive->getQdispPool(); LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } -JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_DEBUG, "~JobQuery"); } +JobQuery::~JobQuery() { + LOGS(_log, LOG_LVL_DEBUG, "~JobQuery"); + LOGS(_log, LOG_LVL_WARN, "~JobQuery QID=" << _idStr); +} /** Attempt to run the job on a worker. * @return - false if it can not setup the job or the maximum number of attempts has been reached. */ -bool JobQuery::runJob() { - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); +bool JobQuery::runJob() { // TODO:UJ delete + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_DEBUG, " runJob " << *this); auto executive = _executive.lock(); if (executive == nullptr) { LOGS(_log, LOG_LVL_ERROR, "runJob failed executive==nullptr"); + return false; } bool superfluous = executive->isLimitRowComplete(); bool cancelled = executive->getCancelled(); bool handlerReset = _jobDescription->respHandler()->reset(); if (!(cancelled || superfluous) && handlerReset) { - auto criticalErr = [this, &executive](std::string const& msg) { + auto criticalErr = [this, &executive](string const& msg) { LOGS(_log, LOG_LVL_ERROR, msg << " " << _jobDescription << " Canceling user query!"); executive->squash(); // This should kill all jobs in this user query. }; LOGS(_log, LOG_LVL_DEBUG, "runJob checking attempt=" << _jobDescription->getAttemptCount()); - std::lock_guard lock(_rmutex); + lock_guard lock(_rmutex); if (_jobDescription->getAttemptCount() < _getMaxAttempts()) { bool okCount = _jobDescription->incrAttemptCount(); if (!okCount) { @@ -101,10 +108,10 @@ bool JobQuery::runJob() { // whether or not we are in SSI as cancellation handling differs. // LOGS(_log, LOG_LVL_TRACE, "runJob calls StartQuery()"); - std::shared_ptr jq(shared_from_this()); + JobQuery::Ptr jq(dynamic_pointer_cast(shared_from_this())); _inSsi = true; if (executive->startQuery(jq)) { - _jobStatus->updateInfo(_idStr, JobStatus::REQUEST, "EXEC"); + _jobStatus->updateInfo(_idStr, qmeta::JobStatus::REQUEST, "EXEC"); return true; } _inSsi = false; @@ -116,10 +123,10 @@ bool JobQuery::runJob() { /// Cancel response handling. Return true if this is the first time cancel has been called. bool JobQuery::cancel(bool superfluous) { - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_DEBUG, "JobQuery::cancel()"); if (_cancelled.exchange(true) == false) { - std::lock_guard lock(_rmutex); + lock_guard lock(_rmutex); // If _inSsi is true then this query request has been passed to SSI and // _queryRequestPtr cannot be a nullptr. Cancellation is complicated. bool cancelled = false; @@ -133,7 +140,7 @@ bool JobQuery::cancel(bool superfluous) { } } if (!cancelled) { - std::ostringstream os; + ostringstream os; os << _idStr << " cancel QueryRequest=" << _queryRequestPtr; LOGS(_log, LOG_LVL_DEBUG, os.str()); if (!superfluous) { @@ -144,7 +151,7 @@ bool JobQuery::cancel(bool superfluous) { LOGS(_log, LOG_LVL_ERROR, " can't markComplete cancelled, executive == nullptr"); return false; } - executive->markCompleted(getIdInt(), false); + executive->markCompleted(getJobId(), false); } if (!superfluous) { _jobDescription->respHandler()->processCancel(); @@ -160,7 +167,7 @@ bool JobQuery::cancel(bool superfluous) { /// cancelling all the jobs that it makes a difference. If either the executive, /// or the job has cancelled, proceeding is probably not a good idea. bool JobQuery::isQueryCancelled() { - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); auto exec = _executive.lock(); if (exec == nullptr) { LOGS(_log, LOG_LVL_WARN, "_executive == nullptr"); @@ -169,8 +176,48 @@ bool JobQuery::isQueryCancelled() { return exec->getCancelled(); } -std::ostream& operator<<(std::ostream& os, JobQuery const& jq) { - return os << "{" << jq.getIdStr() << jq._jobDescription << " " << *jq._jobStatus << "}"; +bool JobQuery::_setUberJobId(UberJobId ujId) { + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); + if (_uberJobId >= 0 && ujId != _uberJobId) { + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " couldn't change UberJobId as ujId=" << ujId << " is owned by " << _uberJobId); + return false; + } + _uberJobId = ujId; + return true; +} + +bool JobQuery::unassignFromUberJob(UberJobId ujId) { + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); + std::lock_guard lock(_rmutex); + if (_uberJobId < 0) { + LOGS(_log, LOG_LVL_INFO, __func__ << " UberJobId already unassigned. attempt by ujId=" << ujId); + return true; + } + if (_uberJobId != ujId) { + LOGS(_log, LOG_LVL_ERROR, + __func__ << " couldn't change UberJobId as ujId=" << ujId << " is owned by " << _uberJobId); + return false; + } + _uberJobId = -1; + + auto exec = _executive.lock(); + // Do not increase the count as it should have been increased when the job was started. + _jobDescription->incrAttemptCountScrubResultsJson(exec, false); + return true; +} + +int JobQuery::getAttemptCount() const { + std::lock_guard lock(_rmutex); + return _jobDescription->getAttemptCount(); +} + +string const& JobQuery::getPayload() const { return _jobDescription->payload(); } + +void JobQuery::callMarkCompleteFunc(bool success) { _markCompleteFunc->operator()(success); } + +ostream& JobQuery::dumpOS(ostream& os) const { + return os << "{" << getIdStr() << _jobDescription << " " << _jobStatus << "}"; } } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index f23544c212..a11b628d49 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -33,6 +33,7 @@ // Qserv headers #include "qdisp/Executive.h" +#include "qdisp/JobBase.h" #include "qdisp/JobDescription.h" #include "qdisp/ResponseHandler.h" #include "util/InstanceCount.h" @@ -42,31 +43,37 @@ namespace lsst::qserv::qdisp { class QdispPool; class QueryRequest; -/** This class is used to describe, monitor, and control a single query to a worker. - * - */ -class JobQuery : public std::enable_shared_from_this { +/// This class is used to describe, monitor, and control a single query to a worker. +/// TODO:UJ once all Jobs are sent out as UberJobs, the purpose of this class is a bit +/// vague. It's components should probably be split between UberJob and +/// JobDescription. +class JobQuery : public JobBase { public: typedef std::shared_ptr Ptr; /// Factory function to make certain a shared_ptr is used and _setup is called. static JobQuery::Ptr create(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - JobStatus::Ptr const& jobStatus, + qmeta::JobStatus::Ptr const& jobStatus, std::shared_ptr const& markCompleteFunc, QueryId qid) { - Ptr jq = std::make_shared(executive, jobDescription, jobStatus, markCompleteFunc, qid); + Ptr jq = Ptr(new JobQuery(executive, jobDescription, jobStatus, markCompleteFunc, qid)); jq->_setup(); return jq; } virtual ~JobQuery(); - virtual bool runJob(); - QueryId getQueryId() const { return _qid; } - int getIdInt() const { return _jobDescription->id(); } - std::string const& getIdStr() const { return _idStr; } + /// Run this job. + bool runJob(); + + QueryId getQueryId() const override { return _qid; } + JobId getJobId() const override { return _jobDescription->id(); } + std::string const& getPayload() const override; + std::string const& getIdStr() const override { return _idStr; } + std::shared_ptr getRespHandler() override { return _jobDescription->respHandler(); } + bool getScanInteractive() const override { return _jobDescription->getScanInteractive(); } JobDescription::Ptr getDescription() { return _jobDescription; } - std::shared_ptr getRespHandler() { return _jobDescription->respHandler(); } - JobStatus::Ptr getStatus() { return _jobStatus; } + + qmeta::JobStatus::Ptr getStatus() override { return _jobStatus; } void setQueryRequest(std::shared_ptr const& qr) { std::lock_guard lock(_rmutex); @@ -77,34 +84,61 @@ class JobQuery : public std::enable_shared_from_this { return _queryRequestPtr; } - std::shared_ptr getMarkCompleteFunc() { return _markCompleteFunc; } + void callMarkCompleteFunc(bool success) override; bool cancel(bool superfluous = false); - bool isQueryCancelled(); + bool isQueryCancelled() override; - Executive::Ptr getExecutive() { return _executive.lock(); } + std::shared_ptr getExecutive() override { return _executive.lock(); } - std::shared_ptr getQdispPool() { return _qdispPool; } + std::shared_ptr getQdispPool() override { return _qdispPool; } - friend std::ostream& operator<<(std::ostream& os, JobQuery const& jq); + std::ostream& dumpOS(std::ostream& os) const override; /// Make a copy of the job description. JobQuery::_setup() must be called after creation. /// Do not call this directly, use create. JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - JobStatus::Ptr const& jobStatus, std::shared_ptr const& markCompleteFunc, - QueryId qid); + qmeta::JobStatus::Ptr const& jobStatus, + std::shared_ptr const& markCompleteFunc, QueryId qid); - bool isCancelled() { return _cancelled; } + /// If the UberJob is unassigned, change the _uberJobId to ujId. + bool setUberJobId(UberJobId ujId) { + std::lock_guard lock(_rmutex); + return _setUberJobId(ujId); + } -protected: - void _setup() { _jobDescription->respHandler()->setJobQuery(shared_from_this()); } + UberJobId getUberJobId() const { + std::lock_guard lock(_rmutex); + return _getUberJobId(); + } - int _getRunAttemptsCount() const { + bool isInUberJob() const { std::lock_guard lock(_rmutex); - return _jobDescription->getAttemptCount(); + return _isInUberJob(); } - int _getMaxAttempts() const { return 5; } // Arbitrary value until solid value with reason determined. - int _getAttemptSleepSeconds() const { return 30; } // As above or until added to config file. + + int getAttemptCount() const; + + /// If ujId is the current owner, clear ownership. + /// @return true if job is unassigned. + bool unassignFromUberJob(UberJobId ujId); + +protected: + void _setup() { + JobBase::Ptr jbPtr = shared_from_this(); + _jobDescription->respHandler()->setJobQuery(jbPtr); + } + + /// @return true if _uberJobId was set, it can only be set if it is unassigned + /// or by the current owner. + /// NOTE: _rmutex must be held before calling this + bool _setUberJobId(UberJobId ujId); + + /// NOTE: _rmutex must be held before calling this + UberJobId _getUberJobId() const { return _uberJobId; } + + /// NOTE: _rmutex must be held before calling this + bool _isInUberJob() const { return _uberJobId >= 0; } // Values that don't change once set. std::weak_ptr _executive; @@ -113,14 +147,16 @@ class JobQuery : public std::enable_shared_from_this { std::shared_ptr _markCompleteFunc; // JobStatus has its own mutex. - JobStatus::Ptr _jobStatus; ///< Points at status in Executive::_statusMap + qmeta::JobStatus::Ptr _jobStatus; ///< Points at status in Executive::_statusMap QueryId const _qid; // User query id std::string const _idStr; ///< Identifier string for logging. // Values that need mutex protection + // TODO:UJ recursive can probably go away with as well as _inSsi. mutable std::recursive_mutex _rmutex; ///< protects _jobDescription, - ///< _queryRequestPtr, and _inSsi + ///< _queryRequestPtr, _uberJobId, + ///< and _inSsi // SSI items std::shared_ptr _queryRequestPtr; @@ -130,6 +166,13 @@ class JobQuery : public std::enable_shared_from_this { std::atomic _cancelled{false}; ///< Lock to make sure cancel() is only called once. std::shared_ptr _qdispPool; + + /// The UberJobId that this job is assigned to. Values less than zero + /// indicate this job is unassigned. To prevent race conditions, + /// an UberJob may only unassign a job if it has the same ID as + /// _uberJobId. + /// All jobs must be unassigned before they can be reassigned. + UberJobId _uberJobId = -1; }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 7c1fbc4efd..c6912f415b 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -42,7 +42,7 @@ class ResponseSummary; namespace lsst::qserv::qdisp { -class JobQuery; +class JobBase; /// ResponseHandler is an interface that handles result bytes. Tasks are /// submitted to an Executive instance naming a resource unit (what resource is @@ -57,7 +57,7 @@ class ResponseHandler { typedef std::shared_ptr Ptr; ResponseHandler() {} - void setJobQuery(std::shared_ptr const& jobQuery) { _jobQuery = jobQuery; } + void setJobQuery(std::shared_ptr const& jobBase) { _jobBase = jobBase; } virtual ~ResponseHandler() {} /// Process a request for pulling and merging a job result into the result table @@ -65,6 +65,17 @@ class ResponseHandler { /// @return true if successful (no error) virtual bool flush(proto::ResponseSummary const& responseSummary) = 0; + /// Collect result data from the worker and merge it with the query result table. + /// @return success - true if the operation was successful + /// @return shouldCancel - if success was false, this being true indicates there + /// was an unrecoverable error in table writing and the query + /// should be cancelled. + virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) = 0; + + /// Add the error to the error output if it is the first error. + virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; + /// Signal an unrecoverable error condition. No further calls are expected. virtual void errorFlush(std::string const& msg, int code) = 0; @@ -84,7 +95,7 @@ class ResponseHandler { std::weak_ptr getJobQuery() { return _jobQuery; } private: - std::weak_ptr _jobQuery; + std::weak_ptr _jobBase; }; inline std::ostream& operator<<(std::ostream& os, ResponseHandler const& r) { return r.print(os); } diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc new file mode 100644 index 0000000000..8b092a6d3b --- /dev/null +++ b/src/qdisp/UberJob.cc @@ -0,0 +1,473 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY + * or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "qdisp/UberJob.h" + +// System headers +#include + +// Third-party headers +#include +#include "nlohmann/json.hpp" + +// Qserv headers +#include "cconfig/CzarConfig.h" +#include "global/LogContext.h" +#include "http/Client.h" +#include "http/MetaModule.h" +#include "proto/ProtoImporter.h" +#include "proto/worker.pb.h" +#include "qdisp/JobQuery.h" +#include "qmeta/JobStatus.h" +#include "util/Bug.h" +#include "util/common.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.UberJob"); +} + +namespace lsst { namespace qserv { namespace qdisp { + +UberJob::Ptr UberJob::create(Executive::Ptr const& executive, + std::shared_ptr const& respHandler, int queryId, int uberJobId, + qmeta::CzarId czarId, + czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData) { + UberJob::Ptr uJob(new UberJob(executive, respHandler, queryId, uberJobId, czarId, workerData)); + uJob->_setup(); + return uJob; +} + +UberJob::UberJob(Executive::Ptr const& executive, std::shared_ptr const& respHandler, + int queryId, int uberJobId, qmeta::CzarId czarId, + czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData) + : JobBase(), + _executive(executive), + _respHandler(respHandler), + _queryId(queryId), + _uberJobId(uberJobId), + _czarId(czarId), + _idStr("QID=" + to_string(_queryId) + ":uj=" + to_string(uberJobId)), + _qdispPool(executive->getQdispPool()), + _workerData(workerData) {} + +void UberJob::_setup() { + JobBase::Ptr jbPtr = shared_from_this(); + _respHandler->setJobQuery(jbPtr); +} + +bool UberJob::addJob(JobQuery::Ptr const& job) { + bool success = false; + if (job->setUberJobId(getJobId())) { + lock_guard lck(_jobsMtx); + _jobs.push_back(job); + success = true; + } + if (!success) { + // TODO:UJ not really the right thing to do, but high visibility wanted for now. + throw util::Bug(ERR_LOC, string("job already in UberJob job=") + job->dump() + " uberJob=" + dump()); + } + return success; +} + +bool UberJob::runUberJob() { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + // Build the uberjob payload for each job. + nlohmann::json uj; + unique_lock jobsLock(_jobsMtx); + auto exec = _executive.lock(); + for (auto const& jqPtr : _jobs) { + jqPtr->getDescription()->incrAttemptCountScrubResultsJson(exec, true); + } + + // Send the uberjob to the worker + auto const method = http::Method::POST; + string const url = "http://" + _wContactInfo->wHost + ":" + to_string(_wContactInfo->wPort) + "/queryjob"; + vector const headers = {"Content-Type: application/json"}; + auto const& czarConfig = cconfig::CzarConfig::instance(); + // See xrdsvc::httpWorkerCzarModule::_handleQueryJob for json message parsing. + json request = {{"version", http::MetaModule::version}, + {"instance_id", czarConfig->replicationInstanceId()}, + {"auth_key", czarConfig->replicationAuthKey()}, + {"worker", _wContactInfo->wId}, + {"czar", + {{"name", czarConfig->name()}, + {"id", czarConfig->id()}, + {"management-port", czarConfig->replicationHttpPort()}, + {"management-host-name", util::get_current_host_fqdn()}}}, + {"uberjob", + {{"queryid", _queryId}, + {"uberjobid", _uberJobId}, + {"czarid", _czarId}, + {"jobs", json::array()}}}}; + + auto& jsUberJob = request["uberjob"]; + auto& jsJobs = jsUberJob["jobs"]; + for (auto const& jbPtr : _jobs) { + auto const description = jbPtr->getDescription(); + if (description == nullptr) { + throw util::Bug(ERR_LOC, cName(__func__) + " description=null for job=" + jbPtr->getIdStr()); + } + auto const jsForWorker = jbPtr->getDescription()->getJsForWorker(); + if (jsForWorker == nullptr) { + throw util::Bug(ERR_LOC, cName(__func__) + " jsForWorker=null for job=" + jbPtr->getIdStr()); + } + json jsJob = {{"jobdesc", *jsForWorker}}; + jsJobs.push_back(jsJob); + jbPtr->getDescription()->resetJsForWorker(); // no longer needed. + } + jobsLock.unlock(); // unlock so other _jobsMtx threads can advance while this waits for transmit + + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << request); + string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << " czarPost url=" << url << " request=" << request.dump() + << " headers=" << headers[0]); + http::Client client(method, url, request.dump(), headers); + bool transmitSuccess = false; + string exceptionWhat; + try { + json const response = client.readAsJson(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " response success=0"); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + exceptionWhat = ex.what(); + } + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure, try to send jobs elsewhere"); + _unassignJobs(); // locks _jobsMtx + setStatusIfOk(qmeta::JobStatus::RESPONSE_ERROR, + cName(__func__) + " not transmitSuccess " + exceptionWhat); + + } else { + setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx + } + return false; +} + +void UberJob::prepScrubResults() { + // TODO:UJ There's a good chance this will not be needed as incomplete files (partitions) + // will not be merged so you don't have to worry about removing rows from incomplete + // jobs or uberjobs from the result table. + throw util::Bug(ERR_LOC, + "TODO:UJ If needed, should call prepScrubResults for all JobQueries in the UberJob "); +} + +void UberJob::_unassignJobs() { + lock_guard lck(_jobsMtx); + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " exec is null"); + return; + } + for (auto&& job : _jobs) { + string jid = job->getIdStr(); + if (!job->unassignFromUberJob(getJobId())) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not unassign job=" << jid << " cancelling"); + exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "unable to re-assign " + jid, + util::ErrorCode::INTERNAL); + exec->squash(); + return; + } + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); + } + _jobs.clear(); + bool const setFlag = true; + exec->setFlagFailedUberJob(setFlag); +} + +bool UberJob::isQueryCancelled() { + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " _executive == nullptr"); + return true; // Safer to assume the worst. + } + return exec->getCancelled(); +} + +bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg) { + // must be locked _jobsMtx + auto currentState = _jobStatus->getState(); + // Setting the same state twice indicates that the system is trying to do something it + // has already done, so doing it a second time would be an error. + if (newState <= currentState) { + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " could not change from state=" << _jobStatus->stateStr(currentState) + << " to " << _jobStatus->stateStr(newState)); + return false; + } + + // Overwriting errors is probably not a good idea. + if (currentState >= qmeta::JobStatus::CANCEL && currentState < qmeta::JobStatus::COMPLETE) { + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " already error current=" << _jobStatus->stateStr(currentState) + << " new=" << _jobStatus->stateStr(newState)); + return false; + } + + _jobStatus->updateInfo(getIdStr(), newState, msg); + for (auto&& jq : _jobs) { + jq->getStatus()->updateInfo(jq->getIdStr(), newState, msg); + } + return true; +} + +void UberJob::callMarkCompleteFunc(bool success) { + LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); + + lock_guard lck(_jobsMtx); + // Need to set this uberJob's status, however exec->markCompleted will set + // the status for each job when it is called. + string source = string("UberJob_") + (success ? "SUCCESS" : "FAILED"); + _jobStatus->updateInfo(getIdStr(), qmeta::JobStatus::COMPLETE, source); + for (auto&& job : _jobs) { + string idStr = job->getIdStr(); + if (success) { + job->getStatus()->updateInfo(idStr, qmeta::JobStatus::COMPLETE, source); + } else { + job->getStatus()->updateInfoNoErrorOverwrite(idStr, qmeta::JobStatus::RESULT_ERROR, source, + util::ErrorCode::INTERNAL, "UberJob_failure"); + } + auto exec = _executive.lock(); + exec->markCompleted(job->getJobId(), success); + } + + // No longer need these here. Executive should still have copies. + _jobs.clear(); +} + +/// Retrieve and process a result file using the file-based protocol +/// Uses a copy of JobQuery::Ptr instead of _jobQuery as a call to cancel() would reset _jobQuery. +json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); + + if (isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); + return _importResultError(true, "cancelled", "Query cancelled"); + } + + auto exec = _executive.lock(); + if (exec == nullptr || exec->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive or cancelled"); + return _importResultError(true, "cancelled", "Query cancelled - no executive"); + } + + if (exec->isLimitRowComplete()) { + int dataIgnored = exec->incrDataIgnoredCount(); + if ((dataIgnored - 1) % 1000 == 0) { + LOGS(_log, LOG_LVL_INFO, + "UberJob ignoring, enough rows already " + << "dataIgnored=" << dataIgnored); + } + return _importResultError(false, "rowLimited", "Enough rows already"); + } + + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileSize=" << fileSize); + + bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); + if (!statusSet) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " setStatusFail could not set status to RESPONSE_READY"); + return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); + } + + JobBase::Ptr jBaseThis = shared_from_this(); + weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); + + // TODO:UJ lambda may not be the best way to do this, alsocheck synchronization - may need a mutex for + // merging. + auto fileCollectFunc = [ujThis, fileUrl, rowCount](util::CmdData*) { + auto ujPtr = ujThis.lock(); + if (ujPtr == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, + "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + return; + } + uint64_t resultRows = 0; + auto [flushSuccess, flushShouldCancel] = + ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); + LOGS(_log, LOG_LVL_DEBUG, ujPtr->cName(__func__) << "::fileCollectFunc"); + if (!flushSuccess) { + // This would probably indicate malformed file+rowCount or + // writing the result table failed. + ujPtr->_importResultError(flushShouldCancel, "mergeError", "merging failed"); + } + + // At this point all data for this job have been read, there's no point in + // having XrdSsi wait for anything. + ujPtr->_importResultFinish(resultRows); + }; + + auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); + exec->queueFileCollect(cmd); + + // If the query meets the limit row complete complete criteria, it will start + // squashing superfluous results so the answer can be returned quickly. + + json jsRet = {{"success", 1}, {"errortype", ""}, {"note", "queued for collection"}}; + return jsRet; +} + +json UberJob::workerError(int errorCode, string const& errorMsg) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " errcode=" << errorCode << " errmsg=" << errorMsg); + + bool const deleteData = true; + bool const keepData = !deleteData; + auto exec = _executive.lock(); + if (exec == nullptr || isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); + return _workerErrorFinish(deleteData, "cancelled"); + } + + if (exec->isLimitRowComplete()) { + int dataIgnored = exec->incrDataIgnoredCount(); + if ((dataIgnored - 1) % 1000 == 0) { + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " ignoring, enough rows already " + << "dataIgnored=" << dataIgnored); + } + return _workerErrorFinish(keepData, "none", "limitRowComplete"); + } + + // Currently there are no detectable recoverable errors from workers. The only + // error that a worker could send back that may possibly be recoverable would + // be a missing table error, which is not trivial to detect. A worker local + // database error may also qualify. + // TODO:UJ see if recoverable errors can be detected on the workers, or + // maybe allow a single retry before sending the error back to the user? + bool recoverableError = false; + recoverableError = true; // TODO:UJ delete after testing + if (recoverableError) { + // The czar should have new maps before the the new UberJob(s) for + // these Jobs are created. (see Czar::_monitor) + _unassignJobs(); + } else { + // Get the error message to the user and kill the user query. + int errState = util::ErrorCode::MYSQLEXEC; + getRespHandler()->flushHttpError(errorCode, errorMsg, errState); + exec->addMultiError(errorCode, errorMsg, errState); + exec->squash(); + } + + string errType = to_string(errorCode) + ":" + errorMsg; + return _workerErrorFinish(deleteData, errType, ""); +} + +json UberJob::_importResultError(bool shouldCancel, string const& errorType, string const& note) { + json jsRet = {{"success", 0}, {"errortype", errorType}, {"note", note}}; + // In all cases, the worker should delete the file as this czar will not ask for it. + + auto exec = _executive.lock(); + if (exec != nullptr) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " shouldCancel=" << shouldCancel << " errorType=" << errorType << " " + << note); + if (shouldCancel) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failing jobs"); + callMarkCompleteFunc(false); // all jobs failed, no retry + exec->squash(); + } else { + /// - each JobQuery in _jobs needs to be flagged as needing to be + /// put in an UberJob and it's attempt count increased and checked + /// against the attempt limit. + /// - executive needs to be told to make new UberJobs until all + /// JobQueries are being handled by an UberJob. + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " reassigning jobs"); + _unassignJobs(); + exec->assignJobsToUberJobs(); + } + } else { + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " already cancelled shouldCancel=" << shouldCancel + << " errorType=" << errorType << " " << note); + } + return jsRet; +} + +nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + /// If this is called, the file has been collected and the worker should delete it + /// + /// This function should call markComplete for all jobs in the uberjob + /// and return a "success:1" json message to be sent to the worker. + bool const statusSet = + setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); + if (!statusSet) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " failed to set status " << getIdStr()); + return {{"success", 0}, {"errortype", "statusMismatch"}, {"note", "failed to set status"}}; + } + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); + return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + } + + bool const success = true; + callMarkCompleteFunc(success); // sets status to COMPLETE + exec->addResultRows(resultRows); + exec->checkLimitRowComplete(); + + json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; + return jsRet; +} + +nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, + std::string const& note) { + // If this is called, the file has been collected and the worker should delete it + // + // Should this call markComplete for all jobs in the uberjob??? + // TODO:UJ Only recoverable errors would be: communication failure, or missing table ??? + // Return a "success:1" json message to be sent to the worker. + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); + return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + } + + json jsRet = {{"success", 1}, {"deletedata", deleteData}, {"errortype", ""}, {"note", ""}}; + return jsRet; +} + +std::ostream& UberJob::dumpOS(std::ostream& os) const { + os << "(jobs sz=" << _jobs.size() << "("; + lock_guard lockJobsMtx(_jobsMtx); + for (auto const& job : _jobs) { + JobDescription::Ptr desc = job->getDescription(); + ResourceUnit ru = desc->resource(); + os << ru.db() << ":" << ru.chunk() << ","; + } + os << "))"; + return os; +} + +}}} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h new file mode 100644 index 0000000000..0dd2f69cdc --- /dev/null +++ b/src/qdisp/UberJob.h @@ -0,0 +1,173 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_QDISP_UBERJOB_H +#define LSST_QSERV_QDISP_UBERJOB_H + +// System headers + +// Qserv headers +#include "qmeta/types.h" +#include "czar/CzarChunkMap.h" // Need nested class. TODO:UJ Make non-nested? +#include "czar/CzarRegistry.h" // Need nested class. TODO:UJ Make non-nested? +#include "qdisp/JobBase.h" +#include "qmeta/JobStatus.h" + +// This header declarations +namespace lsst::qserv::qdisp { + +class JobQuery; + +class QueryRequest; + +/// This class is a contains x number of jobs that need to go to the same worker +/// from a single user query, and contact information for the worker. It also holds +/// some information common to all jobs. +/// The UberJob constructs the message to send to the worker and handles collecting +/// and merging the results. +/// When this UberJobCompletes, all the Jobs it contains are registered as completed. +/// If this UberJob fails, it will be destroyed, un-assigning all of its Jobs. +/// Those Jobs will need to be reassigned to new UberJobs, or the query cancelled. +class UberJob : public JobBase { +public: + using Ptr = std::shared_ptr; + + static Ptr create(std::shared_ptr const& executive, + std::shared_ptr const& respHandler, int queryId, int uberJobId, + qmeta::CzarId czarId, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); + + UberJob() = delete; + UberJob(UberJob const&) = delete; + UberJob& operator=(UberJob const&) = delete; + + virtual ~UberJob(){}; + + bool addJob(std::shared_ptr const& job); + bool runUberJob(); + + std::string cName(const char* funcN) const { return std::string("UberJob::") + funcN + " " + getIdStr(); } + + QueryId getQueryId() const override { return _queryId; } + UberJobId getJobId() const override { + return _uberJobId; + } // TODO:UJ change name when JobBase no longer needed. + std::string const& getIdStr() const override { return _idStr; } + std::shared_ptr getQdispPool() override { return _qdispPool; } // TODO:UJ relocate to JobBase + std::string const& getPayload() const override { return _payload; } // TODO:UJ delete when possible. + std::shared_ptr getRespHandler() override { return _respHandler; } + std::shared_ptr getStatus() override { + return _jobStatus; + } // TODO:UJ relocate to JobBase + bool getScanInteractive() const override { return false; } ///< UberJobs are never interactive. + bool isQueryCancelled() override; // TODO:UJ relocate to JobBase + void callMarkCompleteFunc(bool success) override; ///< call markComplete for all jobs in this UberJob. + std::shared_ptr getExecutive() override { return _executive.lock(); } + + void setQueryRequest(std::shared_ptr const& qr) override { + ; // Do nothing as QueryRequest is only needed for xrootd. TODO:UJ delete function. + } + + /// Return false if not ok to set the status to newState, otherwise set the state for + /// this UberJob and all jobs it contains to newState. + /// This is used both to set status and prevent the system from repeating operations + /// that have already happened. If it returns false, the thread calling this + /// should stop processing. + bool setStatusIfOk(qmeta::JobStatus::State newState, std::string const& msg) { + std::lock_guard jobLock(_jobsMtx); + return _setStatusIfOk(newState, msg); + } + + int getJobCount() const { return _jobs.size(); } + + /// TODO:UJ may not need, + void prepScrubResults(); + + /// Set the worker information needed to send messages to the worker believed to + /// be responsible for the chunks handled in this UberJob. + void setWorkerContactInfo(czar::CzarRegistry::WorkerContactInfo::Ptr const& wContactInfo) { + _wContactInfo = wContactInfo; + } + + /// Get the data for the worker that should handle this UberJob. + czar::CzarChunkMap::WorkerChunksData::Ptr getWorkerData() { return _workerData; } + + /// Collect and merge the results from the worker. + nlohmann::json importResultFile(std::string const& fileUrl, uint64_t rowCount, uint64_t fileSize); + + /// Handle an error from the worker. + nlohmann::json workerError(int errorCode, std::string const& errorMsg); + + std::ostream& dumpOS(std::ostream& os) const override; + +private: + UberJob(std::shared_ptr const& executive, std::shared_ptr const& respHandler, + int queryId, int uberJobId, qmeta::CzarId czarId, + czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); + + /// Used to setup elements that can't be done in the constructor. + void _setup(); + + /// @see setStatusIfOk + /// note: _jobsMtx must be locked before calling. + bool _setStatusIfOk(qmeta::JobStatus::State newState, std::string const& msg); + + /// unassign all Jobs in this UberJob and set the Executive flag to indicate that Jobs need + /// reassignment. + void _unassignJobs(); + + /// Import and error from trying to collect results. + /// TODO:UJ The strings for errorType should have a centralized location in the code - global or util + nlohmann::json _importResultError(bool shouldCancel, std::string const& errorType, + std::string const& note); + + /// Let the executive know that all Jobs in UberJob are complete. + nlohmann::json _importResultFinish(uint64_t resultRows); + + /// Let the Executive know about errors while handling results. + nlohmann::json _workerErrorFinish(bool successful, std::string const& errorType = std::string(), + std::string const& note = std::string()); + + std::vector> _jobs; ///< List of Jobs in this UberJob. + mutable std::mutex _jobsMtx; ///< Protects _jobs, _jobStatus + std::atomic _started{false}; + qmeta::JobStatus::Ptr _jobStatus{new qmeta::JobStatus()}; // TODO:UJ Maybe the JobStatus class should be + // changed to better represent UberJobs + + std::string _payload; ///< XrdSsi message to be sent to the _workerResource. TODO:UJ remove when possible + + std::weak_ptr _executive; + std::shared_ptr _respHandler; + QueryId const _queryId; + UberJobId const _uberJobId; + qmeta::CzarId const _czarId; + + std::string const _idStr; + std::shared_ptr _qdispPool; // TODO:UJ remove when possible. + + // Map of workerData + czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // TODO:UJ this may not be needed + + // Contact information for the target worker. + czar::CzarRegistry::WorkerContactInfo::Ptr _wContactInfo; +}; + +} // namespace lsst::qserv::qdisp + +#endif // LSST_QSERV_QDISP_UBERJOB_H diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 5f69481158..5a7ffc730f 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -40,12 +40,12 @@ #include "global/ResourceUnit.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" -#include "qdisp/MessageStore.h" #include "qdisp/QueryRequest.h" #include "qdisp/SharedResources.h" #include "qdisp/XrdSsiMocks.h" #include "qmeta/QProgress.h" #include "qmeta/QProgressHistory.h" +#include "qmeta/MessageStore.h" #include "qproc/ChunkQuerySpec.h" #include "qproc/TaskMsgFactory.h" #include "util/threadSafe.h" @@ -72,7 +72,15 @@ class MockTaskMsgFactory : public TaskMsgFactory { int attemptCount, qmeta::CzarId czarId, std::ostream& os) override { os << mockPayload; } + + std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, std::string const& chunkResultName, + QueryId queryId, int jobId, int attemptCount, + qmeta::CzarId czarId) override { + return jsPtr; + } + std::string mockPayload; + std::shared_ptr jsPtr; }; } // namespace lsst::qserv::qproc @@ -154,7 +162,7 @@ class SetupTest { std::string qrMsg; std::string str; qdisp::ExecutiveConfig::Ptr conf; - std::shared_ptr ms; + std::shared_ptr ms; qdisp::QdispPool::Ptr qdispPool; qdisp::SharedResources::Ptr sharedResources; qdisp::Executive::Ptr ex; @@ -166,7 +174,7 @@ class SetupTest { qdisp::XrdSsiServiceMock::Reset(); str = qdisp::ExecutiveConfig::getMockStr(); conf = std::make_shared(str, 0); // No updating of QMeta. - ms = std::make_shared(); + ms = std::make_shared(); qdispPool = std::make_shared(true); sharedResources = qdisp::SharedResources::create(qdispPool); @@ -209,7 +217,7 @@ BOOST_AUTO_TEST_CASE(Executive) { LOGS_DEBUG("jobs=1"); tEnv.ex->join(); LOGS_DEBUG("Executive single query test checking"); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qdisp::JobStatus::COMPLETE); + BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::COMPLETE); BOOST_CHECK(tEnv.ex->getEmpty() == true); } @@ -254,7 +262,7 @@ BOOST_AUTO_TEST_CASE(Executive) { BOOST_AUTO_TEST_CASE(MessageStore) { LOGS_DEBUG("MessageStore test start"); - qdisp::MessageStore ms; + qmeta::MessageStore ms; BOOST_CHECK(ms.messageCount() == 0); ms.addMessage(123, "EXECUTIVE", 456, "test1"); std::string str("test2"); @@ -262,7 +270,7 @@ BOOST_AUTO_TEST_CASE(MessageStore) { ms.addMessage(86, "EXECUTIVE", -12, "test3"); BOOST_CHECK(ms.messageCount() == 3); BOOST_CHECK(ms.messageCount(-12) == 2); - qdisp::QueryMessage qm = ms.getMessage(1); + qmeta::QueryMessage qm = ms.getMessage(1); BOOST_CHECK(qm.chunkId == 124 && qm.code == -12 && str.compare(qm.description) == 0); LOGS_DEBUG("MessageStore test end"); } @@ -276,7 +284,7 @@ BOOST_AUTO_TEST_CASE(QueryRequest) { SequentialInt sequence(0); tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); tEnv.ex->join(); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qdisp::JobStatus::RESULT_ERROR); + BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::RESULT_ERROR); BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() > 1); // Retried, eh? BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == qdisp::XrdSsiServiceMock::getReqCount()); } @@ -289,7 +297,7 @@ BOOST_AUTO_TEST_CASE(QueryRequest) { SequentialInt sequence(0); tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); tEnv.ex->join(); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qdisp::JobStatus::RESULT_ERROR); + BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::RESULT_ERROR); BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == 1); } @@ -302,7 +310,7 @@ BOOST_AUTO_TEST_CASE(QueryRequest) { tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); tEnv.ex->join(); LOGS_DEBUG("tEnv.jqTest->...state = " << tEnv.jqTest->getStatus()->getInfo().state); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qdisp::JobStatus::RESULT_ERROR); + BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::RESULT_ERROR); BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == 1); // No retries! } @@ -319,7 +327,7 @@ BOOST_AUTO_TEST_CASE(QueryRequest) { tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); tEnv.ex->join(); BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == - qdisp::JobStatus::COMPLETE); + qmeta::JobStatus::COMPLETE); BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == 1); } */ diff --git a/src/qhttp/Server.cc b/src/qhttp/Server.cc index 5116255cb6..0cb23ae7ea 100644 --- a/src/qhttp/Server.cc +++ b/src/qhttp/Server.cc @@ -277,7 +277,31 @@ std::shared_ptr Server::_findPathHandler(Request::Ptr reque for (auto& pathHandler : pathHandlersIt->second) { if (boost::regex_match(request->path, pathMatch, pathHandler.path.regex)) { pathHandler.path.updateParamsFromMatch(request, pathMatch); +#if 0 // &&& <<<<<<< HEAD return std::make_shared(pathHandler); +#else // &&& ===== + LOGLS_DEBUG(_log, logger(this) << logger(request->_socket) << "invoking handler for " + << pathHandler.path.regex); + try { + pathHandler.handler(request, response); + } catch (boost::system::system_error const& e) { + LOGLS_ERROR(_log, logger(this) << logger(request->_socket) + << "exception thrown from handler: " << e.what()); + switch (e.code().value()) { + case errc::permission_denied: + response->sendStatus(STATUS_FORBIDDEN); + break; + default: + response->sendStatus(STATUS_INTERNAL_SERVER_ERR); + break; + } + } catch (std::exception const& e) { + LOGLS_ERROR(_log, logger(this) << logger(request->_socket) + << "exception thrown from handler: " << e.what()); + response->sendStatus(STATUS_INTERNAL_SERVER_ERR); + } + return; +#endif //&&& >>>>>>> ca9f7b24f (Added some error handling.) } } } diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index f0b307fef0..969971f5ed 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -1,6 +1,8 @@ add_library(qmeta SHARED) -target_sources(qmeta PRIVATE +target_sources(qserv_meta PRIVATE + JobStatus.cc + MessageStore.cc QMeta.cc QMetaMysql.cc QMetaSelect.cc @@ -18,6 +20,8 @@ target_link_libraries(qmeta PUBLIC qdisp log util + mysqlclient_r + http ) install(TARGETS qmeta) diff --git a/src/qmeta/Exceptions.h b/src/qmeta/Exceptions.h index deeccf73b7..511e1cd357 100644 --- a/src/qmeta/Exceptions.h +++ b/src/qmeta/Exceptions.h @@ -109,6 +109,15 @@ class MissingTableError : public QMetaError { virtual std::string typeName() const override { return "MissingTableError"; } }; +/// Exception thrown when the specified metadata table is empty. +class EmptyTableError : public QMetaError { +public: + EmptyTableError(util::Issue::Context const& ctx, std::string const& table) + : QMetaError(ctx, "Query metadata table is empty: " + table) {} + + virtual std::string typeName() const override { return "EmptyTableError"; } +}; + /// Exception thrown when database consistency is violated. class ConsistencyError : public QMetaError { public: diff --git a/src/qdisp/JobStatus.cc b/src/qmeta/JobStatus.cc similarity index 74% rename from src/qdisp/JobStatus.cc rename to src/qmeta/JobStatus.cc index 20dad135a3..bf31305499 100644 --- a/src/qdisp/JobStatus.cc +++ b/src/qmeta/JobStatus.cc @@ -33,7 +33,7 @@ */ // Class header -#include "qdisp/JobStatus.h" +#include "qmeta/JobStatus.h" // System headers #include @@ -44,18 +44,43 @@ #include "lsst/log/Log.h" namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobStatus"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.qmeta.JobStatus"); } -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { JobStatus::Info::Info() : state(UNKNOWN), stateCode(0) { stateTime = getNow(); } void JobStatus::updateInfo(std::string const& idMsg, JobStatus::State s, std::string const& source, int code, std::string const& desc, MessageSeverity severity) { std::lock_guard lock(_mutex); + _updateInfo(idMsg, s, source, code, desc, severity); +} + +void JobStatus::_updateInfo(std::string const& idMsg, JobStatus::State s, std::string const& source, int code, + std::string const& desc, MessageSeverity severity) { + LOGS(_log, LOG_LVL_DEBUG, + idMsg << " Updating state to: " << s << " code=" << code << " " << desc << " src=" << source); + _info.stateTime = getNow(); + _info.state = s; + _info.stateCode = code; + _info.stateDesc = desc; + _info.source = source; + _info.severity = severity; +} + +void JobStatus::updateInfoNoErrorOverwrite(std::string const& idMsg, JobStatus::State s, + std::string const& source, int code, std::string const& desc, + MessageSeverity severity) { + std::lock_guard lock(_mutex); + auto jState = _info.state; + if (jState != qmeta::JobStatus::CANCEL && jState != qmeta::JobStatus::RESPONSE_ERROR && + jState != qmeta::JobStatus::RESULT_ERROR && jState != qmeta::JobStatus::MERGE_ERROR) { + _updateInfo(idMsg, s, source, code, desc, severity); + } - LOGS(_log, LOG_LVL_DEBUG, idMsg << " Updating state to: " << s << " code=" << code << " " << desc); + LOGS(_log, LOG_LVL_DEBUG, + idMsg << " Updating state to: " << s << " code=" << code << " " << desc << " src=" << source); _info.stateTime = getNow(); _info.state = s; _info.stateCode = code; @@ -140,4 +165,4 @@ std::ostream& operator<<(std::ostream& os, JobStatus::Info const& info) { return os; } -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qmeta diff --git a/src/qdisp/JobStatus.h b/src/qmeta/JobStatus.h similarity index 79% rename from src/qdisp/JobStatus.h rename to src/qmeta/JobStatus.h index d693921001..89ecda0c84 100644 --- a/src/qdisp/JobStatus.h +++ b/src/qmeta/JobStatus.h @@ -20,8 +20,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_QDISP_JOBSTATUS_H -#define LSST_QSERV_QDISP_JOBSTATUS_H +#ifndef LSST_QSERV_QMETA_JOBSTATUS_H +#define LSST_QSERV_QMETA_JOBSTATUS_H // System headers #include @@ -34,9 +34,9 @@ // qserv headers #include "global/constants.h" -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { -/** Monitor execution of a chunk query against an SSI ressource +/** Monitor execution of a chunk query. * * JobStatus instances receive timestamped reports of execution State. This * allows a manager object to receive updates on status without exposing its @@ -61,13 +61,14 @@ class JobStatus { UNKNOWN = 0, REQUEST = 1203, RESPONSE_READY, - RESPONSE_ERROR, RESPONSE_DATA, RESPONSE_DATA_NACK, RESPONSE_DONE, + CANCEL, + RESPONSE_ERROR, // Errors must be between CANCEL and COMPLETE RESULT_ERROR, MERGE_ERROR, - CANCEL, + RETRY_ERROR, COMPLETE = 2000 }; @@ -96,6 +97,13 @@ class JobStatus { void updateInfo(std::string const& idMsg, State s, std::string const& source, int code = 0, std::string const& desc = "", MessageSeverity severity = MSG_INFO); + /// Same as updateInfo() except existing error states are not overwritten. + /// @see updateInfo() + /// @return Negative values indicate the status was changed, zero and positive values + void updateInfoNoErrorOverwrite(std::string const& idMsg, State s, std::string const& source, + int code = 0, std::string const& desc = "", + MessageSeverity severity = MSG_INFO); + struct Info { Info(); // More detailed debugging may store a vector of states, appending @@ -116,11 +124,21 @@ class JobStatus { return _info; } + State getState() const { + std::lock_guard lock(_mutex); + return _info.state; + } + static std::string stateStr(JobStatus::State const& state); friend std::ostream& operator<<(std::ostream& os, JobStatus const& es); private: + /// @see updateInfo() + /// note: _mutex must be held before calling. + void _updateInfo(std::string const& idMsg, JobStatus::State s, std::string const& source, int code, + std::string const& desc, MessageSeverity severity); + Info _info; mutable std::mutex _mutex; ///< Mutex to guard concurrent updates }; @@ -128,6 +146,6 @@ std::ostream& operator<<(std::ostream& os, JobStatus const& es); std::ostream& operator<<(std::ostream& os, JobStatus::Info const& inf); std::ostream& operator<<(std::ostream& os, JobStatus::State const& state); -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qmeta -#endif // LSST_QSERV_QDISP_JOBSTATUS_H +#endif // LSST_QSERV_META_JOBSTATUS_H diff --git a/src/qdisp/MessageStore.cc b/src/qmeta/MessageStore.cc similarity index 87% rename from src/qdisp/MessageStore.cc rename to src/qmeta/MessageStore.cc index 784dd847b3..e4e32fe746 100644 --- a/src/qdisp/MessageStore.cc +++ b/src/qmeta/MessageStore.cc @@ -23,7 +23,7 @@ // See MessageStore.h // Class header -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" // System headers #include @@ -36,13 +36,12 @@ // Qserv headers #include "global/constants.h" -#include "qdisp/JobStatus.h" namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.MessageStore"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.qmeta.MessageStore"); } -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { //////////////////////////////////////////////////////////////////////// // public @@ -50,9 +49,9 @@ namespace lsst::qserv::qdisp { void MessageStore::addMessage(int chunkId, std::string const& msgSource, int code, std::string const& description, MessageSeverity severity, - JobStatus::TimeType timestamp) { - if (timestamp == JobStatus::TimeType()) { - timestamp = JobStatus::getNow(); + qmeta::JobStatus::TimeType timestamp) { + if (timestamp == qmeta::JobStatus::TimeType()) { + timestamp = qmeta::JobStatus::getNow(); } auto level = code < 0 ? LOG_LVL_ERROR : LOG_LVL_DEBUG; LOGS(_log, level, "Add msg: " << chunkId << " " << msgSource << " " << code << " " << description); @@ -80,4 +79,4 @@ int MessageStore::messageCount(int code) const { return count; } -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qmeta diff --git a/src/qdisp/MessageStore.h b/src/qmeta/MessageStore.h similarity index 90% rename from src/qdisp/MessageStore.h rename to src/qmeta/MessageStore.h index c42114f01e..7fe9823c55 100644 --- a/src/qdisp/MessageStore.h +++ b/src/qmeta/MessageStore.h @@ -29,8 +29,8 @@ /// The MessageStore classes are responsible for maintaining status and /// error messages associated with a query. -#ifndef LSST_QSERV_QDISP_MESSAGESTORE_H -#define LSST_QSERV_QDISP_MESSAGESTORE_H +#ifndef LSST_QSERV_QMETA_MESSAGESTORE_H +#define LSST_QSERV_QMETA_MESSAGESTORE_H // System headers #include @@ -40,13 +40,13 @@ // Qserv headers #include "global/constants.h" -#include "qdisp/JobStatus.h" +#include "qmeta/JobStatus.h" -namespace lsst::qserv::qdisp { +namespace lsst::qserv::qmeta { struct QueryMessage { QueryMessage(int chunkId_, std::string const& msgSource_, int code_, std::string description_, - JobStatus::TimeType timestamp_, MessageSeverity severity_) + qmeta::JobStatus::TimeType timestamp_, MessageSeverity severity_) : chunkId(chunkId_), msgSource(msgSource_), code(code_), @@ -58,7 +58,7 @@ struct QueryMessage { std::string msgSource; int code; std::string description; - JobStatus::TimeType timestamp; + qmeta::JobStatus::TimeType timestamp; MessageSeverity severity; }; @@ -95,7 +95,7 @@ class MessageStore { */ void addMessage(int chunkId, std::string const& msgSource, int code, std::string const& description, MessageSeverity severity_ = MessageSeverity::MSG_INFO, - JobStatus::TimeType timestamp = JobStatus::TimeType()); + qmeta::JobStatus::TimeType timestamp = qmeta::JobStatus::TimeType()); /** Add an error message to this MessageStore * @@ -117,6 +117,6 @@ class MessageStore { std::vector _queryMessages; }; -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qmeta -#endif // LSST_QSERV_QDISP_MESSAGESTORE_H +#endif // LSST_QSERV_QMETA_MESSAGESTORE_H diff --git a/src/qmeta/QMeta.h b/src/qmeta/QMeta.h index 92fc832d7f..05d54526bb 100644 --- a/src/qmeta/QMeta.h +++ b/src/qmeta/QMeta.h @@ -23,6 +23,7 @@ #define LSST_QSERV_QMETA_QMETA_H // System headers +#include #include #include #include @@ -30,19 +31,63 @@ #include // Qserv headers +#include "global/clock_defs.h" #include "global/intTypes.h" #include "qmeta/QInfo.h" #include "qmeta/types.h" namespace lsst::qserv::qdisp { -class MessageStore; class QueryMessage; } // namespace lsst::qserv::qdisp namespace lsst::qserv::qmeta { +class MessageStore; + /// @addtogroup qmeta +/** + * The structure ChunkMap encapsulates a disposition of chunks at Qserv workers + * along with a time when the map was updated. + * + * Here is an example on how to using the map for getting info on all chunks in + * the given context: + * @code + * std::string const worker = "worker-001"; + * std::string const database = "LSST-DR01"; + * std::string const table = "Object"; + * + * ChunkMap const& chunkMap = ...; + * for (auto const& [chunk, size] : chunkMap[worker][database][table]) { + * ... + * } + * @endcode + */ +struct QMetaChunkMap { + /// @return 'true' if the map is empty (or constructed using the default constructor) + bool empty() const { + return workers.empty() || (std::chrono::time_point() == updateTime); + } + + // NOTE: Separate types were added here for the sake of clarity to avoid + // a definition of the unreadable nested map. + + struct ChunkInfo { + unsigned int chunk = 0; ///< The chunk number + size_t size = 0; ///< The file size (in bytes) of the chunk table + }; + typedef std::vector Chunks; ///< Collection of chunks + typedef std::map Tables; ///< tables-to-chunks + typedef std::map Databases; ///< Databases-to-tables + typedef std::map Workers; ///< Workers-to-databases + + /// The chunk disposition map for all workers. + Workers workers; + + /// The last time the map was updated (since UNIX Epoch). + TIMEPOINT updateTime; +}; + /** * @ingroup qmeta * @brief Interface for query metadata. @@ -265,7 +310,23 @@ class QMeta { virtual void saveResultQuery(QueryId queryId, std::string const& query) = 0; /// Write messages/errors generated during the query to the QMessages table. - virtual void addQueryMessages(QueryId queryId, std::shared_ptr const& msgStore) = 0; + virtual void addQueryMessages(QueryId queryId, std::shared_ptr const& msgStore) = 0; + + /** + * Fetch the chunk map which was updated after the specified time point. + * @param prevUpdateTime The cut off time for the chunk map age. Note that the default + * value of the parameter represents the start time of the UNIX Epoch. Leaving the default + * value forces an attempt to read the map from the database if the one would exist + * in there. + * @return Return the most current chunk disposition or the empty object if the persistent + * map is older than it was requested.The result could be evaluated by calling + * method empty() on the result object. + * @throws EmptyTableError if the corresponding metadata table doesn't have any record + * @throws SqlError for any other error related to MySQL + */ + virtual QMetaChunkMap getChunkMap( + std::chrono::time_point const& prevUpdateTime = + std::chrono::time_point()) = 0; protected: // Default constructor diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index c522d1ab3c..0269824fb1 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -25,6 +25,7 @@ // System headers #include +#include // Third-party headers #include "boost/lexical_cast.hpp" @@ -34,9 +35,10 @@ #include "lsst/log/Log.h" // Qserv headers -#include "qdisp/JobStatus.h" -#include "qdisp/MessageStore.h" +#include "global/stringUtil.h" #include "qmeta/Exceptions.h" +#include "qmeta/JobStatus.h" +#include "qmeta/MessageStore.h" #include "qmeta/QMetaTransaction.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" @@ -733,14 +735,14 @@ void QMetaMysql::saveResultQuery(QueryId queryId, string const& query) { trans->commit(); } -void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr const& msgStore) { +void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr const& msgStore) { int msgCount = msgStore->messageCount(); int cancelCount = 0; int completeCount = 0; int execFailCount = 0; map msgCountMap; for (int i = 0; i != msgCount; ++i) { - qdisp::QueryMessage const& qMsg = msgStore->getMessage(i); + qmeta::QueryMessage const& qMsg = msgStore->getMessage(i); try { _addQueryMessage(queryId, qMsg, cancelCount, completeCount, execFailCount, msgCountMap); } catch (qmeta::SqlError const& ex) { @@ -749,11 +751,11 @@ void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr 0 || execFailCount > 0) { - qdisp::QueryMessage qm(-1, "CANCELTOTAL", 0, + qmeta::QueryMessage qm(-1, "CANCELTOTAL", 0, string("{\"CANCEL_count\":") + to_string(cancelCount) + ", \"EXECFAIL_count\":" + to_string(execFailCount) + ", \"COMPLETE_count\":" + to_string(completeCount) + "}", - qdisp::JobStatus::getNow(), MessageSeverity::MSG_INFO); + qmeta::JobStatus::getNow(), MessageSeverity::MSG_INFO); _addQueryMessage(queryId, qm, cancelCount, completeCount, execFailCount, msgCountMap); } @@ -762,13 +764,95 @@ void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr const& prevUpdateTime) { + lock_guard lock(_dbMutex); + + QMetaChunkMap chunkMap; + + auto trans = QMetaTransaction::create(*_conn); + + // Check if the table needs to be read. Note that the default value of + // the previous update timestamp always forces an attempt to read the map. + auto const updateTime = _getChunkMapUpdateTime(lock); + bool const force = + (prevUpdateTime == chrono::time_point()) || (prevUpdateTime < updateTime); + if (!force) { + trans->commit(); + return QMetaChunkMap(); + } + + // Read the map itself + + sql::SqlErrorObject errObj; + sql::SqlResults results; + + string const tableName = "chunkMap"; + string const query = "SELECT `worker`,`database`,`table`,`chunk`,`size` FROM `" + tableName + "`"; + LOGS(_log, LOG_LVL_DEBUG, "Executing query: " << query); + if (!_conn->runQuery(query, results, errObj)) { + LOGS(_log, LOG_LVL_ERROR, "query failed: " << query); + throw SqlError(ERR_LOC, errObj); + } + vector> const rows = results.extractFirstNColumns(5); + trans->commit(); + + if (rows.empty()) throw EmptyTableError(ERR_LOC, tableName); + try { + for (auto const& row : rows) { + string const& worker = row[0]; + string const& database = row[1]; + string const& table = row[2]; + unsigned int chunk = lsst::qserv::stoui(row[3]); + size_t const size = stoull(row[4]); + chunkMap.workers[worker][database][table].push_back(QMetaChunkMap::ChunkInfo{chunk, size}); + LOGS(_log, LOG_LVL_TRACE, + "QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table + << " chunk=" << chunk << " sz=" << size); + } + chunkMap.updateTime = updateTime; + } catch (exception const& ex) { + string const msg = "Failed to parse result set of query " + query + ", ex: " + string(ex.what()); + throw ConsistencyError(ERR_LOC, msg); + } + return chunkMap; +} + +chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock_guard const& lock) { + sql::SqlErrorObject errObj; + sql::SqlResults results; + string const tableName = "chunkMapStatus"; + string const query = + "SELECT TIME_TO_SEC(`update_time`) FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; + LOGS(_log, LOG_LVL_DEBUG, "Executing query: " << query); + if (!_conn->runQuery(query, results, errObj)) { + LOGS(_log, LOG_LVL_ERROR, "query failed: " << query); + throw SqlError(ERR_LOC, errObj); + } + vector updateTime; + if (!results.extractFirstColumn(updateTime, errObj)) { + LOGS(_log, LOG_LVL_ERROR, "Failed to extract result set of query " + query); + throw SqlError(ERR_LOC, errObj); + } + if (updateTime.empty()) { + throw EmptyTableError(ERR_LOC, tableName); + } else if (updateTime.size() > 1) { + throw ConsistencyError(ERR_LOC, "Too many rows in result set of query " + query); + } + try { + return chrono::time_point() + chrono::seconds(stol(updateTime[0])); + } catch (exception const& ex) { + string const msg = "Failed to parse result set of query " + query + ", ex: " + string(ex.what()); + throw ConsistencyError(ERR_LOC, msg); + } +} + +void QMetaMysql::_addQueryMessage(QueryId queryId, qmeta::QueryMessage const& qMsg, int& cancelCount, int& completeCount, int& execFailCount, map& msgCountMap) { // Don't add duplicate messages. if (qMsg.msgSource == "DUPLICATE") return; @@ -829,7 +913,7 @@ void QMetaMysql::_addQueryMessage(QueryId queryId, qdisp::QueryMessage const& qM query += ", " + to_string(qMsg.code); query += ", \"" + _conn->escapeString(severity) + "\""; query += ", \"" + _conn->escapeString(qMsg.description) + "\""; - query += ", " + to_string(qdisp::JobStatus::timeToInt(qMsg.timestamp)); + query += ", " + to_string(qmeta::JobStatus::timeToInt(qMsg.timestamp)); query += ")"; // run query sql::SqlErrorObject errObj; diff --git a/src/qmeta/QMetaMysql.h b/src/qmeta/QMetaMysql.h index c34d577dc0..0afb534be2 100644 --- a/src/qmeta/QMetaMysql.h +++ b/src/qmeta/QMetaMysql.h @@ -23,6 +23,7 @@ #define LSST_QSERV_QMETA_QMETAMYSQL_H // System headers +#include #include #include #include @@ -41,11 +42,12 @@ class SqlConnection; namespace lsst::qserv::qmeta { +class QueryMessage; + /// @addtogroup qmeta /** * @ingroup qmeta - * * @brief Mysql-based implementation of qserv metadata. */ @@ -244,7 +246,11 @@ class QMetaMysql : public QMeta { void saveResultQuery(QueryId queryId, std::string const& query) override; /// @see QMeta::addQueryMessages() - void addQueryMessages(QueryId queryId, std::shared_ptr const& msgStore) override; + void addQueryMessages(QueryId queryId, std::shared_ptr const& msgStore) override; + + /// @see QMeta::getChunkMap + QMetaChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = + std::chrono::time_point()) override; protected: /// Check that all necessary tables exist @@ -260,8 +266,20 @@ class QMetaMysql : public QMeta { }; private: + /** + * Read the last update time of the chunk map. + * @param A lock acquired on the mutex _dbMutex. + * @return The update time + * @throw EmptyTableError If the corrresponding table is epty + * @throw SqlError For any SQL-specific error + * @throw ConsistencyError For any problem met when parsing or interpreting results read + * from the table. + */ + std::chrono::time_point _getChunkMapUpdateTime( + std::lock_guard const& lock); + /// Add qMsg to the permanent message table. - void _addQueryMessage(QueryId queryId, qdisp::QueryMessage const& qMsg, int& cancelCount, + void _addQueryMessage(QueryId queryId, qmeta::QueryMessage const& qMsg, int& cancelCount, int& completeCount, int& execFailCount, std::map& msgCountMap); diff --git a/src/qmeta/testQMeta.cc b/src/qmeta/testQMeta.cc index bc4e19521c..89e05bc979 100644 --- a/src/qmeta/testQMeta.cc +++ b/src/qmeta/testQMeta.cc @@ -37,6 +37,7 @@ #include "qmeta/QMetaMysql.h" #include "qmeta/QProgress.h" #include "qmeta/QProgressData.h" +#include "qmeta/MessageStore.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" #include "sql/SqlErrorObject.h" @@ -377,4 +378,10 @@ BOOST_AUTO_TEST_CASE(messWithQueryStats) { BOOST_CHECK(caught); } +BOOST_AUTO_TEST_CASE(getChunkMap) { + // The test assumes that the underlying tables exists and it's empty. + QMetaChunkMap chunkMap; + BOOST_CHECK_THROW(qMeta->getChunkMap(), EmptyTableError); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/src/qproc/TaskMsgFactory.cc b/src/qproc/TaskMsgFactory.cc index ac7a5afe09..8a2d7434dc 100644 --- a/src/qproc/TaskMsgFactory.cc +++ b/src/qproc/TaskMsgFactory.cc @@ -36,6 +36,7 @@ #include // Third-party headers +#include "nlohmann/json.hpp" // LSST headers #include "lsst/log/Log.h" @@ -51,8 +52,70 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qproc.TaskMsgFactory"); } +using namespace std; + namespace lsst::qserv::qproc { +// TODO:UJ - Probaly just delete this +bool TaskMsgFactory::fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& chunkQuerySpec, + std::string const& chunkResultName, QueryId queryId, int jobId, + int attemptCount, qmeta::CzarId czarId) { + std::string resultTable("Asdfasfd"); + if (!chunkResultName.empty()) { + resultTable = chunkResultName; + } + // shared + taskMsg->set_db(chunkQuerySpec.db); + taskMsg->set_queryid(queryId); + taskMsg->set_jobid(jobId); + taskMsg->set_attemptcount(attemptCount); + taskMsg->set_czarid(czarId); + + // scanTables (for shared scans) + // check if more than 1 db in scanInfo + std::string db; + for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { + if (db.empty()) { + db = sTbl.db; + } + } + + for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { + lsst::qserv::proto::TaskMsg_ScanTable* msgScanTbl = taskMsg->add_scantable(); + sTbl.copyToScanTable(msgScanTbl); + } + + taskMsg->set_scanpriority(chunkQuerySpec.scanInfo.scanRating); + taskMsg->set_scaninteractive(chunkQuerySpec.scanInteractive); + + // per-chunk + taskMsg->set_chunkid(chunkQuerySpec.chunkId); + // per-fragment + // TODO refactor to simplify + if (chunkQuerySpec.nextFragment.get()) { + ChunkQuerySpec const* sPtr = &chunkQuerySpec; + while (sPtr) { + LOGS(_log, LOG_LVL_TRACE, "nextFragment"); + for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { + LOGS(_log, LOG_LVL_TRACE, (sPtr->queries).at(t)); + } + // Linked fragments will not have valid subChunkTables vectors, + // So, we reuse the root fragment's vector. + _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, + sPtr->queries); + sPtr = sPtr->nextFragment.get(); + } + } else { + LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); + for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { + LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); + } + _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, + chunkQuerySpec.queries); + } + return true; +} + std::shared_ptr TaskMsgFactory::_makeMsg(ChunkQuerySpec const& chunkQuerySpec, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId) { @@ -147,4 +210,99 @@ void TaskMsgFactory::serializeMsg(ChunkQuerySpec const& s, std::string const& ch m->SerializeToOstream(&os); } +std::shared_ptr TaskMsgFactory::makeMsgJson(ChunkQuerySpec const& chunkQuerySpec, + std::string const& chunkResultName, + QueryId queryId, int jobId, int attemptCount, + qmeta::CzarId czarId) { + std::string resultTable("Asdfasfd"); + if (!chunkResultName.empty()) { + resultTable = chunkResultName; + } + + // TODO:UJ verify that these can be put in the uberjob to reduce duplicates + // and the size of the message. + auto jsJobMsgPtr = std::shared_ptr( + new nlohmann::json({{"czarId", czarId}, + {"queryId", queryId}, + {"jobId", jobId}, + {"attemptCount", attemptCount}, + {"querySpecDb", chunkQuerySpec.db}, + {"scanPriority", chunkQuerySpec.scanInfo.scanRating}, + {"scanInteractive", chunkQuerySpec.scanInteractive}, + {"maxTableSize", (cconfig::CzarConfig::instance()->getMaxTableSizeMB())}, + {"chunkScanTables", nlohmann::json::array()}, + {"chunkId", chunkQuerySpec.chunkId}, + {"queryFragments", nlohmann::json::array()}})); + + auto& jsJobMsg = *jsJobMsgPtr; + + auto& chunkScanTables = jsJobMsg["chunkScanTables"]; + for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { + nlohmann::json cst = {{"db", sTbl.db}, + {"table", sTbl.table}, + {"lockInMemory", sTbl.lockInMemory}, + {"tblScanRating", sTbl.scanRating}}; + chunkScanTables.push_back(move(cst)); + } + + auto& jsFragments = jsJobMsg["queryFragments"]; + if (chunkQuerySpec.nextFragment.get()) { + ChunkQuerySpec const* sPtr = &chunkQuerySpec; + while (sPtr) { + LOGS(_log, LOG_LVL_TRACE, "nextFragment"); + for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { + LOGS(_log, LOG_LVL_DEBUG, __func__ << " q=" << (sPtr->queries).at(t)); + } + for (auto const& sbi : sPtr->subChunkIds) { + LOGS(_log, LOG_LVL_DEBUG, __func__ << " sbi=" << sbi); + } + // Linked fragments will not have valid subChunkTables vectors, + // So, we reuse the root fragment's vector. + _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, + sPtr->queries); + sPtr = sPtr->nextFragment.get(); + } + } else { + LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); + for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { + LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); + } + _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, + chunkQuerySpec.queries); + } + + return jsJobMsgPtr; +} + +void TaskMsgFactory::_addFragmentJson(nlohmann::json& jsFragments, std::string const& resultName, + DbTableSet const& subChunkTables, std::vector const& subchunkIds, + std::vector const& queries) { + nlohmann::json jsFrag = {{"resultTable", resultName}, + {"queries", nlohmann::json::array()}, + {"subchunkTables", nlohmann::json::array()}, + {"subchunkIds", nlohmann::json::array()}}; + + auto& jsQueries = jsFrag["queries"]; + for (auto& qry : queries) { + nlohmann::json jsQry = {{"subQuery", qry}}; + jsQueries.push_back(move(jsQry)); + } + + // Add the db+table pairs to the subchunk. + auto& jsSubchunkTables = jsFrag["subchunkTables"]; + for (auto& tbl : subChunkTables) { + nlohmann::json jsSubchunkTbl = {{"scDb", tbl.db}, {"scTable", tbl.table}}; + jsSubchunkTables.push_back(move(jsSubchunkTbl)); + LOGS(_log, LOG_LVL_TRACE, "added dbtbl=" << tbl.db << "." << tbl.table); + } + + // Add subchunk id numbers + auto& jsSubchunkIds = jsFrag["subchunkIds"]; + for (auto& subchunkId : subchunkIds) { + jsSubchunkIds.push_back(subchunkId); + } + + jsFragments.push_back(move(jsFrag)); +} + } // namespace lsst::qserv::qproc diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h index dc2d0ed130..d770d2c5c4 100644 --- a/src/qproc/TaskMsgFactory.h +++ b/src/qproc/TaskMsgFactory.h @@ -35,6 +35,9 @@ #include #include +// Third party headers +#include "nlohmann/json.hpp" + // Qserv headers #include "global/DbTable.h" #include "global/intTypes.h" @@ -58,14 +61,31 @@ class TaskMsgFactory { virtual void serializeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId, std::ostream& os); + /// Use the provided information to fill in taskMsg. + /// @return true if successful. + bool fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& s, std::string const& chunkResultName, + QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId); + + /// Make and return the json message for a single Job. + virtual std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, + std::string const& chunkResultName, QueryId queryId, + int jobId, int attemptCount, qmeta::CzarId czarId); + private: + // TODO:UJ delete when possible std::shared_ptr _makeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId); + // TODO:UJ delete when possible void _addFragment(proto::TaskMsg& taskMsg, std::string const& resultName, DbTableSet const& subChunkTables, std::vector const& subChunkIds, std::vector const& queries); + + /// Make a json message for a single fragment. + void _addFragmentJson(nlohmann::json& jsFragments, std::string const& resultName, + DbTableSet const& subChunkTables, std::vector const& subChunkIds, + std::vector const& queries); }; } // namespace lsst::qserv::qproc diff --git a/src/replica/apps/MasterControllerHttpApp.cc b/src/replica/apps/MasterControllerHttpApp.cc index 797b22b2ef..d366318f51 100644 --- a/src/replica/apps/MasterControllerHttpApp.cc +++ b/src/replica/apps/MasterControllerHttpApp.cc @@ -166,6 +166,10 @@ MasterControllerHttpApp::MasterControllerHttpApp(int argc, char* argv[]) " This affect replicas to be deleted from the workers during the synchronization" " stages.", _forceQservSync); + parser().flag("qserv-chunk-map-update", + "The flag which would result in updating the chunk disposition map" + " in Qserv's QMeta database.", + _qservChunkMapUpdate); parser().flag("purge", "The binary flag which, if provided, enables the 'purge' algorithm in" " the end of each replication cycle that eliminates excess replicas which" @@ -217,7 +221,7 @@ int MasterControllerHttpApp::runImpl() { _replicationTask = ReplicationTask::create( _controller, [self](Task::Ptr const& ptr) { self->_isFailed.fail(); }, _qservSyncTimeoutSec, - _disableQservSync, _forceQservSync, _replicationIntervalSec, _purge); + _disableQservSync, _forceQservSync, _qservChunkMapUpdate, _replicationIntervalSec, _purge); _replicationTask->start(); _healthMonitorTask = HealthMonitorTask::create( diff --git a/src/replica/apps/MasterControllerHttpApp.h b/src/replica/apps/MasterControllerHttpApp.h index 1b9fc8f39f..836917eae7 100644 --- a/src/replica/apps/MasterControllerHttpApp.h +++ b/src/replica/apps/MasterControllerHttpApp.h @@ -146,6 +146,7 @@ class MasterControllerHttpApp : public Application { bool _purge; bool _disableQservSync; bool _forceQservSync; + bool _qservChunkMapUpdate; bool _permanentDelete; /// A connection URL for the MySQL service of the Qserv master database. diff --git a/src/replica/contr/ReplicationTask.cc b/src/replica/contr/ReplicationTask.cc index 99091b3e4a..a5d40bbfcd 100644 --- a/src/replica/contr/ReplicationTask.cc +++ b/src/replica/contr/ReplicationTask.cc @@ -22,24 +22,35 @@ // Class header #include "replica/contr/ReplicationTask.h" +// System headers +#include + // Qserv headers +#include "replica/config/Configuration.h" #include "replica/jobs/FindAllJob.h" #include "replica/jobs/FixUpJob.h" #include "replica/jobs/ReplicateJob.h" #include "replica/jobs/RebalanceJob.h" #include "replica/jobs/PurgeJob.h" +#include "replica/mysql/DatabaseMySQL.h" +#include "replica/mysql/DatabaseMySQLGenerator.h" +#include "replica/mysql/DatabaseMySQLUtils.h" +#include "replica/services/DatabaseServices.h" +#include "replica/util/ReplicaInfo.h" using namespace std; namespace lsst::qserv::replica { +using namespace database::mysql; + ReplicationTask::Ptr ReplicationTask::create(Controller::Ptr const& controller, Task::AbnormalTerminationCallbackType const& onTerminated, unsigned int qservSyncTimeoutSec, bool disableQservSync, - bool forceQservSync, unsigned int replicationIntervalSec, - bool purge) { + bool forceQservSync, bool qservChunkMapUpdate, + unsigned int replicationIntervalSec, bool purge) { return Ptr(new ReplicationTask(controller, onTerminated, qservSyncTimeoutSec, disableQservSync, - forceQservSync, replicationIntervalSec, purge)); + forceQservSync, qservChunkMapUpdate, replicationIntervalSec, purge)); } bool ReplicationTask::onRun() { @@ -52,6 +63,8 @@ bool ReplicationTask::onRun() { launch(priority, saveReplicaInfo, allWorkers); if (!_disableQservSync) sync(_qservSyncTimeoutSec, _forceQservSync); + if (_qservChunkMapUpdate) _updateChunkMap(); + launch(priority); if (!_disableQservSync) sync(_qservSyncTimeoutSec, _forceQservSync); @@ -74,11 +87,82 @@ bool ReplicationTask::onRun() { ReplicationTask::ReplicationTask(Controller::Ptr const& controller, Task::AbnormalTerminationCallbackType const& onTerminated, unsigned int qservSyncTimeoutSec, bool disableQservSync, bool forceQservSync, - unsigned int replicationIntervalSec, bool purge) + bool qservChunkMapUpdate, unsigned int replicationIntervalSec, bool purge) : Task(controller, "REPLICATION-THREAD ", onTerminated, replicationIntervalSec), _qservSyncTimeoutSec(qservSyncTimeoutSec), _disableQservSync(disableQservSync), _forceQservSync(forceQservSync), + _qservChunkMapUpdate(qservChunkMapUpdate), _purge(purge) {} +void ReplicationTask::_updateChunkMap() { + // Open MySQL connection using the RAII-style handler that would automatically + // abort the transaction should any problem occured when loading data into the table. + ConnectionHandler h; + try { + h.conn = Connection::open(Configuration::qservCzarDbParams("qservMeta")); + } catch (exception const& ex) { + error("failed to connect to the czar's database server, ex: " + string(ex.what())); + return; + } + QueryGenerator const g(h.conn); + + // Get info on known chunk replicas from the persistent store of the Replication system + // and package those into ready-to-ingest data. + bool const allDatabases = true; + string const emptyDatabaseFilter; + bool const isPublished = true; + bool const includeFileInfo = true; // need this to access tables sizes + vector rows; + for (auto const& workerName : serviceProvider()->config()->workers()) { + vector replicas; + serviceProvider()->databaseServices()->findWorkerReplicas(replicas, workerName, emptyDatabaseFilter, + allDatabases, isPublished, includeFileInfo); + for (auto const& replica : replicas) { + for (auto const& fileInfo : replica.fileInfo()) { + if (fileInfo.isData() && !fileInfo.isOverlap()) { + rows.push_back(g.packVals(workerName, replica.database(), fileInfo.baseTable(), + replica.chunk(), fileInfo.size)); + } + } + } + } + if (rows.empty()) { + warn("no replicas found in the persistent state of the Replication system"); + return; + } + + // Get the limit for the length of the bulk insert queries. The limit is needed + // to run the query generation. + size_t maxQueryLength = 0; + string const globalVariableName = "max_allowed_packet"; + try { + string const query = g.showVars(SqlVarScope::GLOBAL, globalVariableName); + h.conn->executeInOwnTransaction([&query, &maxQueryLength](auto conn) { + bool const noMoreThanOne = true; + if (!selectSingleValue(conn, query, maxQueryLength, "Value", noMoreThanOne)) { + throw runtime_error("no such variable found"); + } + }); + } catch (exception const& ex) { + error("failed to get a value of GLOBAL '" + globalVariableName + "', ex: " + string(ex.what())); + return; + } + + // Execute a sequence of queries atomically + vector const deleteQueries = {g.delete_("chunkMap"), g.delete_("chunkMapStatus")}; + vector insertQueries = g.insertPacked( + "chunkMap", g.packIds("worker", "database", "table", "chunk", "size"), rows, maxQueryLength); + insertQueries.push_back(g.insert("chunkMapStatus", Sql::NOW)); + try { + h.conn->executeInOwnTransaction([&deleteQueries, &insertQueries](auto conn) { + for (auto const& query : deleteQueries) conn->execute(query); + for (auto const& query : insertQueries) conn->execute(query); + }); + } catch (exception const& ex) { + error("failed to update chunk map in the Czar database, ex: " + string(ex.what())); + return; + } +} + } // namespace lsst::qserv::replica diff --git a/src/replica/contr/ReplicationTask.h b/src/replica/contr/ReplicationTask.h index dabd74bc41..2fb3c73bc4 100644 --- a/src/replica/contr/ReplicationTask.h +++ b/src/replica/contr/ReplicationTask.h @@ -56,6 +56,7 @@ class ReplicationTask : public Task { * up on the Qserv synchronization requests. * @param disableQservSync Disable replica synchronization at Qserv workers if 'true'. * @param forceQservSync Force chunk removal at worker resource collections if 'true'. + * @param qservChunkMapUpdate Update the chunk disposition map in Qserv's QMeta database if 'true'. * @param replicationIntervalSec The number of seconds to wait in the end of each * iteration loop before to begin the new one. * @param purge Purge excess replicas if 'true'. @@ -64,7 +65,7 @@ class ReplicationTask : public Task { static Ptr create(Controller::Ptr const& controller, Task::AbnormalTerminationCallbackType const& onTerminated, unsigned int qservSyncTimeoutSec, bool disableQservSync, bool forceQservSync, - unsigned int replicationIntervalSec, bool purge); + bool qservChunkMapUpdate, unsigned int replicationIntervalSec, bool purge); protected: /// @see Task::onRun() @@ -74,7 +75,9 @@ class ReplicationTask : public Task { /// @see ReplicationTask::create() ReplicationTask(Controller::Ptr const& controller, AbnormalTerminationCallbackType const& onTerminated, unsigned int qservSyncTimeoutSec, bool disableQservSync, bool forceQservSync, - unsigned int replicationIntervalSec, bool purge); + bool qservChunkMapUpdate, unsigned int replicationIntervalSec, bool purge); + + void _updateChunkMap(); /// The maximum number of seconds to be waited before giving up /// on the Qserv synchronization requests. @@ -82,6 +85,8 @@ class ReplicationTask : public Task { bool const _disableQservSync; ///< Disable replica synchroization at Qserv workers if 'true'. bool const _forceQservSync; ///< Force removal at worker resource collections if 'true'. + bool const + _qservChunkMapUpdate; /// Update the chunk disposition map in Qserv's QMeta database if 'true'. bool const _purge; ///< Purge excess replicas if 'true'. }; diff --git a/src/replica/contr/Task.h b/src/replica/contr/Task.h index 6efee40f25..4b5b853c28 100644 --- a/src/replica/contr/Task.h +++ b/src/replica/contr/Task.h @@ -198,6 +198,12 @@ class Task : public EventLogger, public std::enable_shared_from_this { */ void debug(std::string const& msg) { LOGS(_log, LOG_LVL_DEBUG, context() << msg); } + /** + * Log a message into the Logger's LOG_LVL_WARN stream. + * @param msg A message to be logged. + */ + void warn(std::string const& msg) { LOGS(_log, LOG_LVL_WARN, context() << msg); } + /** * Log a message into the Logger's LOG_LVL_ERROR stream. * @param msg A message to be logged. diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 2e83c022f6..c02e7f7107 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -161,7 +161,7 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { void InfileMerger::mergeCompleteFor(int jobId) { std::lock_guard resultSzLock(_mtxResultSizeMtx); - _totalResultSize += _perJobResultSize[jobId]; + _totalResultSize += _perJobResultSize[jobId]; // TODO:UJ this can probably be simplified } bool InfileMerger::merge(proto::ResponseSummary const& resp, @@ -228,6 +228,117 @@ bool InfileMerger::merge(proto::ResponseSummary const& resp, return ret; } +bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { + UberJobId const uJobId = uberJob->getJobId(); + std::string queryIdJobStr = uberJob->getIdStr(); + if (!_queryIdStrSet) { + _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); + } + + // Nothing to do if size is zero. + if (responseData.row_size() == 0) { + return true; + } + + // Do nothing if the query got cancelled for any reason. + if (uberJob->isQueryCancelled()) { + return true; + } + auto executive = uberJob->getExecutive(); + if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { + return true; + } + + std::unique_ptr semaLock; + if (_dbEngine != MYISAM) { + // needed for parallel merging with INNODB and MEMORY + semaLock.reset(new util::SemaLock(*_semaMgrConn)); + } + + TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, + bool success) { + if (!success) return; + if (std::chrono::duration const seconds = end - start; seconds.count() > 0) { + qdisp::CzarStats::get()->addXRootDSSIRecvRate(bytes / seconds.count()); + } + }; + auto tct = make_shared>(cbf); + + bool ret = false; + // Add columns to rows in virtFile. + util::Timer virtFileT; + virtFileT.start(); + // UberJobs only get one attempt + int resultJobId = makeJobIdAttempt(uberJob->getJobId(), 0); + ProtoRowBuffer::Ptr pRowBuffer = std::make_shared( + responseData, resultJobId, _jobIdColName, _jobIdSqlType, _jobIdMysqlType); + std::string const virtFile = _infileMgr.prepareSrc(pRowBuffer); + std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); + virtFileT.stop(); + + // If the job attempt is invalid, exit without adding rows. + // It will wait here if rows need to be deleted. + if (_invalidJobAttemptMgr.incrConcurrentMergeCount(resultJobId)) { + return true; + } + + size_t const resultSize = responseData.transmitsize(); + size_t tResultSize; + { + std::lock_guard resultSzLock(_mtxResultSizeMtx); + _perJobResultSize[uJobId] += resultSize; + tResultSize = _totalResultSize + _perJobResultSize[uJobId]; + } + if (tResultSize > _maxResultTableSizeBytes) { + std::ostringstream os; + os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable + << " is too large at " << tResultSize << " bytes, max allowed size is " << _maxResultTableSizeBytes + << " bytes"; + LOGS(_log, LOG_LVL_ERROR, os.str()); + _error = util::Error(-1, os.str(), -1); + return false; + } + + tct->addToValue(resultSize); + tct->setSuccess(); + tct.reset(); // stop transmit recieve timer before merging happens. + + qdisp::CzarStats::get()->addTotalBytesRecv(resultSize); + qdisp::CzarStats::get()->addTotalRowsRecv(responseData.rowcount()); + + // Stop here (if requested) after collecting stats on the amount of data collected + // from workers. + if (_config.debugNoMerge) { + return true; + } + + auto start = std::chrono::system_clock::now(); + switch (_dbEngine) { + case MYISAM: + ret = _applyMysqlMyIsam(infileStatement, resultSize); + break; + case INNODB: // Fallthrough + case MEMORY: + ret = _applyMysqlInnoDb(infileStatement, resultSize); + break; + default: + throw std::invalid_argument("InfileMerger::_dbEngine is unknown =" + engineToStr(_dbEngine)); + } + auto end = std::chrono::system_clock::now(); + auto mergeDur = std::chrono::duration_cast(end - start); + LOGS(_log, LOG_LVL_DEBUG, + "mergeDur=" << mergeDur.count() << " sema(total=" << _semaMgrConn->getTotalCount() + << " used=" << _semaMgrConn->getUsedCount() << ")"); + if (not ret) { + LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); + } + _invalidJobAttemptMgr.decrConcurrentMergeCount(); + + LOGS(_log, LOG_LVL_DEBUG, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); + + return ret; +} + bool InfileMerger::_applyMysqlMyIsam(std::string const& query, size_t resultSize) { std::unique_lock lock(_mysqlMutex); for (int j = 0; !_mysqlConn.connected(); ++j) { diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index fa2c65d88f..6d2d98fef2 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -54,7 +54,11 @@ class ResponseSummary; } // namespace proto namespace qdisp { class MessageStore; +class UberJob; } // namespace qdisp +namespace QMeta { +class MessageStore; +} namespace qproc { class DatabaseModels; } @@ -105,6 +109,9 @@ class InfileMerger { /// @return true if merge was successfully imported. bool merge(proto::ResponseSummary const& resp, std::shared_ptr const& csvStream); + /// Merge the result data collected over Http. + bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + /// Indicate the merge for the job is complete. void mergeCompleteFor(int jobId); diff --git a/src/sql/SqlResults.cc b/src/sql/SqlResults.cc index eee4a1de6d..b96d5d1c85 100644 --- a/src/sql/SqlResults.cc +++ b/src/sql/SqlResults.cc @@ -179,6 +179,24 @@ bool SqlResults::extractFirst4Columns(std::vector& col1, std::vecto return true; } +std::vector> SqlResults::extractFirstNColumns(size_t numColumns) { + std::vector> rows; + for (int resultIdx = 0, numResults = _results.size(); resultIdx < numResults; ++resultIdx) { + MYSQL_ROW row; + while ((row = mysql_fetch_row(_results[resultIdx])) != nullptr) { + std::vector columns; + columns.reserve(numColumns); + for (size_t colIdx = 0; colIdx < numColumns; ++colIdx) { + columns.push_back(row[colIdx]); + } + rows.push_back(std::move(columns)); + } + mysql_free_result(_results[resultIdx]); + } + _results.clear(); + return rows; +} + bool SqlResults::extractFirstValue(std::string& ret, SqlErrorObject& errObj) { if (_results.size() != 1) { std::stringstream ss; diff --git a/src/sql/SqlResults.h b/src/sql/SqlResults.h index a48f5a78a1..2c860c6213 100644 --- a/src/sql/SqlResults.h +++ b/src/sql/SqlResults.h @@ -97,12 +97,21 @@ class SqlResults : boost::noncopyable { std::vector&, std::vector&, SqlErrorObject&); bool extractFirst4Columns(std::vector&, std::vector&, std::vector&, std::vector&, SqlErrorObject&); + template bool extractFirstColumns(SqlErrorObject& err, Columns&... cols) { std::vector>> columns = {std::ref(cols)...}; return _extractFirstColumnsImpl(err, columns); } + /** + * Extract a result set into the 2D array. + * @param numColumns The number of columns in the array. + * @return a 2D array, where the first index of the array represents rows + * and the second index represents columns. + */ + std::vector> extractFirstNColumns(size_t numColumns); + void freeResults(); /// Return row iterator diff --git a/src/util/TimeUtils.cc b/src/util/TimeUtils.cc index 17c68c9849..17edb71fa5 100644 --- a/src/util/TimeUtils.cc +++ b/src/util/TimeUtils.cc @@ -55,4 +55,13 @@ uint64_t TimeUtils::tp2ms(chrono::system_clock::time_point const& tp) { return chrono::duration_cast(tp.time_since_epoch()).count(); } +string TimeUtils::timePointToDateTimeString(TIMEPOINT const& point) { + auto const timer = chrono::system_clock::to_time_t(point); + auto broken_time = *localtime(&timer); + + ostringstream ss; + ss << put_time(&broken_time, "%Y-%m-%d %H:%M:%S"); + return ss.str(); +} + } // namespace lsst::qserv::util diff --git a/src/util/TimeUtils.h b/src/util/TimeUtils.h index ac5a31480a..bd3f19ca5b 100644 --- a/src/util/TimeUtils.h +++ b/src/util/TimeUtils.h @@ -31,6 +31,9 @@ #include #include +// Qserv headers +#include "global/clock_defs.h" + // This header declarations namespace lsst::qserv::util { @@ -47,6 +50,9 @@ struct TimeUtils { /// @return a human-readable timestamp in a format 'YYYY-MM-DD HH:MM:SS.mmm' static std::string toDateTimeString(std::chrono::milliseconds const& millisecondsSinceEpoch); + /// @return a human-readable time in a format 'YYYY-MM-DD HH:MM:SS' + static std::string timePointToDateTimeString(TIMEPOINT const& point); + /** * @param tp The timepoint to be converted. * @return The number of milliseconds since UNIX Epoch diff --git a/src/wbase/CMakeLists.txt b/src/wbase/CMakeLists.txt index add5682ee8..7f0365cb73 100644 --- a/src/wbase/CMakeLists.txt +++ b/src/wbase/CMakeLists.txt @@ -6,6 +6,7 @@ target_sources(wbase PRIVATE FileChannelShared.cc SendChannel.cc Task.cc + UberJobData.cc UserQueryInfo.cc WorkerCommand.cc ) diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index d2ad2db475..9ad5f60eed 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -38,6 +38,7 @@ #include "proto/ProtoHeaderWrap.h" #include "proto/worker.pb.h" #include "wbase/Task.h" +#include "wbase/UberJobData.h" #include "wconfig/WorkerConfig.h" #include "wpublish/QueriesAndChunks.h" #include "util/Bug.h" @@ -263,14 +264,47 @@ shared_ptr FileChannelShared::create(shared_ptr const& sendChannel, qmeta::CzarId czarId, string const& workerId) - : _sendChannel(sendChannel), _czarId(czarId), _workerId(workerId) { + : _isUberJob(false), + _sendChannel(sendChannel), + _uberJobId(0), + _czarId(czarId), + _czarHostName(""), ///< Name of the czar host. + _czarPort(-1), + _workerId(workerId), + _protobufArena(make_unique()), + _scsId(scsSeqId++) { LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created"); if (_sendChannel == nullptr) { throw util::Bug(ERR_LOC, "FileChannelShared constructor given nullptr"); } } +FileChannelShared::Ptr FileChannelShared::create(std::shared_ptr const& uberJob, + qmeta::CzarId czarId, string const& czarHostName, + int czarPort, string const& workerId) { + lock_guard const lock(_resultsDirCleanupMtx); + return Ptr(new FileChannelShared(uberJob, czarId, czarHostName, czarPort, workerId)); +} + +FileChannelShared::FileChannelShared(std::shared_ptr const& uberJobData, + qmeta::CzarId czarId, string const& czarHostName, int czarPort, + string const& workerId) + : _isUberJob(true), + _sendChannel(nullptr), + _uberJobData(uberJobData), + _uberJobId(uberJobData->getUberJobId()), + _czarId(czarId), + _czarHostName(czarHostName), + _czarPort(czarPort), + _workerId(workerId), + _protobufArena(make_unique()), + _scsId(scsSeqId++), + _useHttp(true) { + LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created scsId=" << _scsId << " ujId=" << _uberJobId); +} + FileChannelShared::~FileChannelShared() { + LOGS(_log, LOG_LVL_DEBUG, "~FileChannelShared scsId=" << _scsId << " ujId=" << _uberJobId); // Normally, the channel should not be dead at this time. If it's already // dead it means there was a problem to process a query or send back a response // to Czar. In either case, the file would be useless and it has to be deleted @@ -278,13 +312,15 @@ FileChannelShared::~FileChannelShared() { if (isDead()) { _removeFile(lock_guard(_tMtx)); } - if (_sendChannel != nullptr) { - _sendChannel->setDestroying(); - if (!_sendChannel->isDead()) { - _sendChannel->kill("~FileChannelShared()"); + if (!_useHttp) { + if (_sendChannel != nullptr) { + _sendChannel->setDestroying(); + if (!_sendChannel->isDead()) { + _sendChannel->kill("~FileChannelShared()"); + } } } - LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared deleted"); + LOGS(_log, LOG_LVL_DEBUG, "~FileChannelShared end"); } void FileChannelShared::setTaskCount(int taskCount) { _taskCount = taskCount; } @@ -302,8 +338,12 @@ bool FileChannelShared::kill(string const& note) { } bool FileChannelShared::isDead() { - if (_sendChannel == nullptr) return true; - return _sendChannel->isDead(); + if (!_useHttp) { + if (_sendChannel == nullptr) return true; + return _sendChannel->isDead(); + } else { + return _dead; + } } string FileChannelShared::makeIdStr(int qId, int jId) { @@ -314,11 +354,24 @@ string FileChannelShared::makeIdStr(int qId, int jId) { bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared_ptr const& task, bool cancelled) { lock_guard const tMtxLock(_tMtx); - if (!_sendResponse(tMtxLock, task, cancelled, multiErr)) { - LOGS(_log, LOG_LVL_ERROR, "Could not transmit the error message to Czar."); - return false; + if (!_useHttp) { + if (!_sendResponse(tMtxLock, task, cancelled, multiErr)) { + LOGS(_log, LOG_LVL_ERROR, "Could not transmit the error message to Czar."); + return false; + } + return true; + } else { + auto ujData = _uberJobData.lock(); + if (ujData == nullptr) { + LOGS(_log, LOG_LVL_WARN, + __func__ << " not sending error as ujData is null " << multiErr.toString()); + return false; + } + // Delete the result file as nobody will come looking for it. + _kill(tMtxLock, " buildAndTransmitError"); + return ujData->responseError(multiErr, task, cancelled); } - return true; + return false; } bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& task, @@ -414,7 +467,16 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& streamMutexLock, string const& note) { LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared::" << __func__ << " " << note); - return _sendChannel->kill(note); + if (!_useHttp) { + return _sendChannel->kill(note); + } else { + bool oldVal = _dead.exchange(true); + if (!oldVal) { + LOGS(_log, LOG_LVL_WARN, "FileChannelShared first kill call " << note); + } + _removeFile(streamMutexLock); + return oldVal; + } } void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_ptr const& task, @@ -462,16 +524,22 @@ void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p } void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { - if (!_fileName.empty() && _file.is_open()) { - _file.close(); + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared::_removeFile " << _fileName << " scsId=" << _scsId); + if (!_fileName.empty()) { + if (_file.is_open()) { + _file.close(); + } boost::system::error_code ec; + LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared::" << __func__ << " removing " << _fileName); fs::remove_all(fs::path(_fileName), ec); if (ec.value() != 0) { LOGS(_log, LOG_LVL_WARN, "FileChannelShared::" << __func__ << " failed to remove the result file '" << _fileName << "', ec: " << ec << "."); + return; } } + _fileName.clear(); } bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ptr const& task, @@ -493,7 +561,6 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ // Prepare the response object and serialize in into a message that will // be sent to Czar. - proto::ResponseSummary response; response.set_wname(_workerId); response.set_queryid(queryId); @@ -523,20 +590,62 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ LOGS(_log, LOG_LVL_DEBUG, __func__ << " idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size()); - // Send the message sent out-of-band within the SSI metadata. - if (!_sendChannel->setMetadata(_responseBuf.data(), _responseBuf.size())) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed in setMetadata " << idStr); - _kill(streamMutexLock, "setMetadata"); - return false; - } + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&& idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size() << " useHttp=" << _useHttp); + if (!_useHttp) { + proto::ResponseSummary response; + response.set_wname(_workerId); + response.set_queryid(queryId); + response.set_jobid(jobId); + response.set_fileresource_xroot(task->resultFileXrootUrl()); + response.set_fileresource_http(task->resultFileHttpUrl()); + response.set_attemptcount(task->getAttemptCount()); + response.set_rowcount(_rowcount); + response.set_transmitsize(_transmitsize); + string errorMsg; + int errorCode = 0; + if (!multiErr.empty()) { + errorMsg = multiErr.toOneLineString(); + errorCode = multiErr.firstErrorCode(); + } else if (cancelled) { + errorMsg = "cancelled"; + errorCode = -1; + } + if (!errorMsg.empty() or (errorCode != 0)) { + errorMsg = "FileChannelShared::" + string(__func__) + " error(s) in result for chunk #" + + to_string(task->getChunkId()) + ": " + errorMsg; + response.set_errormsg(errorMsg); + response.set_errorcode(errorCode); + LOGS(_log, LOG_LVL_ERROR, errorMsg); + } + response.SerializeToString(&_responseBuf); - // Send back the empty object since no info is expected by a caller - // for this type of requests beyond the usual error notifications (if any). - // Note that this call is needed to initiate the transaction. - if (!_sendChannel->sendData((char const*)0, 0)) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed in sendData " << idStr); - _kill(streamMutexLock, "sendData"); - return false; + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size()); + + // Send the message sent out-of-band within the SSI metadata. + if (!_sendChannel->setMetadata(_responseBuf.data(), _responseBuf.size())) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " failed in setMetadata " << idStr); + _kill(streamMutexLock, "setMetadata"); + return false; + } + + // Send back the empty object since no info is expected by a caller + // for this type of requests beyond the usual error notifications (if any). + // Note that this call is needed to initiate the transaction. + if (!_sendChannel->sendData((char const*)0, 0)) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " failed in sendData " << idStr); + _kill(streamMutexLock, "sendData"); + return false; + } + } else { + auto ujData = _uberJobData.lock(); + if (ujData == nullptr) { + LOGS(_log, LOG_LVL_WARN, __func__ << " uberJobData is nullptr for ujId=" << _uberJobId); + return false; + } + string httpFileUrl = task->resultFileHttpUrl(); + ujData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); } return true; } diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 0ad6fa75d1..1e23d5b8d0 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -49,6 +49,7 @@ class MultiError; } // namespace lsst::qserv::util namespace lsst::qserv::wbase { +class UberJobData; /// The class is responsible for writing mysql result rows as Protobuf /// serialized messages into an output file. Once a task (or all sub-chunk @@ -109,6 +110,11 @@ class FileChannelShared { static Ptr create(std::shared_ptr const& sendChannel, qmeta::CzarId czarId, std::string const& workerId = std::string()); + /// The factory method for handling UberJob over http. + static Ptr create(std::shared_ptr const& uberJob, qmeta::CzarId czarId, + std::string const& czarHostName, int czarPort, + std::string const& workerId); // TODO:UJ delete all params except uberJob + FileChannelShared() = delete; FileChannelShared(FileChannelShared const&) = delete; FileChannelShared& operator=(FileChannelShared const&) = delete; @@ -153,10 +159,14 @@ class FileChannelShared { bool isDead(); private: - /// Private constructor to protect shared pointer integrity. + /// TODO:UJ delete sendchannel version of constructor when possible. FileChannelShared(std::shared_ptr const& sendChannel, qmeta::CzarId czarId, std::string const& workerId); + /// Private constructor to protect shared pointer integrity. + FileChannelShared(std::shared_ptr const& uberJob, qmeta::CzarId czarId, + std::string const& czarHostName, int czarPort, std::string const& workerId); + /// @see wbase::SendChannel::kill /// @param streamMutexLock - Lock on mutex _streamMutex to be acquired before calling the method. bool _kill(std::lock_guard const& streamMutexLock, std::string const& note); @@ -215,9 +225,16 @@ class FileChannelShared { mutable std::mutex _tMtx; ///< Protects data recording and Czar notification + bool _isUberJob; ///< true if this is using UberJob http. To be removed when _sendChannel goes away. + std::shared_ptr const _sendChannel; ///< Used to pass encoded information to XrdSsi. - qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). - std::string const _workerId; ///< The unique identifier of the worker. + std::weak_ptr _uberJobData; ///< Pointer to UberJobData + + UberJobId const _uberJobId; ///< The UberJobId + qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). TODO:UJ delete + std::string const _czarHostName; ///< Name of the czar host. TODO:UJ delete + int const _czarPort; ///< port for the czar. TODO:UJ delete + std::string const _workerId; ///< The unique identifier of the worker. TODO:UJ delete /// streamMutex is used to protect _lastCount and messages that are sent /// using FileChannelShared. @@ -251,6 +268,10 @@ class FileChannelShared { uint32_t _rowcount = 0; ///< The total numnber of rows in all result sets of a query. uint64_t _transmitsize = 0; ///< The total amount of data (bytes) in all result sets of a query. + uint64_t _headerCount = 0; ///< Count of headers received. + + bool const _useHttp = false; ///< to be eliminated when xrootd is no longer used. + std::atomic _dead{false}; ///< Set to true when the contents of the file are no longer useful. }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index b6f86633a1..4d276db677 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -46,6 +46,7 @@ #include "global/constants.h" #include "global/LogContext.h" #include "global/UnsupportedError.h" +#include "http/RequestBody.h" #include "mysql/MySqlConfig.h" #include "proto/worker.pb.h" #include "util/Bug.h" @@ -56,6 +57,7 @@ #include "util/TimeUtils.h" #include "wbase/Base.h" #include "wbase/FileChannelShared.h" +#include "wbase/UberJobData.h" #include "wbase/UserQueryInfo.h" #include "wconfig/WorkerConfig.h" #include "wdb/QueryRunner.h" @@ -63,6 +65,7 @@ using namespace std; using namespace std::chrono_literals; +using namespace nlohmann; namespace fs = boost::filesystem; namespace { @@ -81,6 +84,17 @@ string buildResultFilePath(string const& resultFileName, string const& resultsDi return fs::weakly_canonical(fs::path(resultsDirname) / resultFileName).string(); } +string buildUjResultFilePath(lsst::qserv::wbase::UberJobData::Ptr const& ujData, + string const& resultsDirname) { + if (resultsDirname.empty()) return resultsDirname; + fs::path path(resultsDirname); + // UberJobs have multiple chunks which can each have different attempt numbers. + // However, each CzarID + UberJobId should be unique as UberJobs are not retried. + path /= to_string(ujData->getCzarId()) + "-" + to_string(ujData->getQueryId()) + "-" + + to_string(ujData->getUberJobId()) + "-0" + ".proto"; + return path.string(); +} + size_t const MB_SIZE_BYTES = 1024 * 1024; } // namespace @@ -106,7 +120,6 @@ bool Task::ChunkIdGreater::operator()(Task::Ptr const& x, Task::Ptr const& y) { } string const Task::defaultUser = "qsmaster"; -IdSet Task::allIds{}; TaskScheduler::TaskScheduler() { auto hour = chrono::milliseconds(1h); @@ -137,7 +150,6 @@ Task::Task(TaskMsgPtr const& t, int fragmentNumber, shared_ptr co _attemptCount(t->attemptcount()), _queryFragmentNum(fragmentNumber), _fragmentHasSubchunks(t->fragment(fragmentNumber).has_subchunks()), - _hasDb(t->has_db()), _db(t->has_db() ? t->db() : ""), _czarId(t->has_czarid() ? t->czarid() : -1) { // These attributes will be passed back to Czar in the Protobuf response @@ -152,9 +164,6 @@ Task::Task(TaskMsgPtr const& t, int fragmentNumber, shared_ptr co user = defaultUser; } - allIds.add(to_string(_qId) + "_" + to_string(_jId)); - LOGS(_log, LOG_LVL_DEBUG, "Task(...) " << "this=" << this << " : " << allIds); - // Determine which major tables this task will use. int const size = t->scantable_size(); for (int j = 0; j < size; ++j) { @@ -208,10 +217,81 @@ Task::Task(TaskMsgPtr const& t, int fragmentNumber, shared_ptr co } } -Task::~Task() { - allIds.remove(to_string(_qId) + "_" + to_string(_jId)); - LOGS(_log, LOG_LVL_TRACE, "~Task() : " << allIds); +/// When the constructor is called, there is not enough information +/// available to define the action to take when this task is run, so +/// Command::setFunc() is used set the action later. This is why +/// the util::CommandThreadPool is not called here. +Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, + shared_ptr const& userQueryInfo, size_t templateId, bool hasSubchunks, + int subchunkId, string const& db, proto::ScanInfo const& scanInfo, bool scanInteractive, + int maxTableSize, vector const& fragSubTables, vector const& fragSubchunkIds, + shared_ptr const& sc, uint16_t resultsHttpPort) + : _userQueryInfo(userQueryInfo), + _sendChannel(sc), + _tSeq(++taskSequence), + _qId(ujData->getQueryId()), + _templateId(templateId), + _hasChunkId((chunkId >= 0)), + _chunkId(chunkId), + _subchunkId(subchunkId), + _jId(jobId), + _attemptCount(attemptCount), + _queryFragmentNum(fragmentNumber), + _fragmentHasSubchunks(hasSubchunks), + _db(db), + _czarId(ujData->getCzarId()), + _scanInfo(scanInfo), + _scanInteractive(scanInteractive), + _maxTableSize(maxTableSize * ::MB_SIZE_BYTES) { + // These attributes will be passed back to Czar in the Protobuf response + // to advice which result delivery channel to use. + auto const workerConfig = wconfig::WorkerConfig::instance(); + auto const resultDeliveryProtocol = workerConfig->resultDeliveryProtocol(); + _resultFilePath = ::buildUjResultFilePath(ujData, workerConfig->resultsDirname()); + auto const fqdn = util::get_current_host_fqdn(); + if (resultDeliveryProtocol == wconfig::ConfigValResultDeliveryProtocol::HTTP) { + // TODO:UJ it seems like this should just be part of the FileChannelShared??? + _resultFileHttpUrl = "http://" + fqdn + ":" + to_string(resultsHttpPort) + _resultFilePath; + } else { + throw runtime_error("wbase::Task::Task: unsupported results delivery protocol: " + + wconfig::ConfigValResultDeliveryProtocol::toString(resultDeliveryProtocol)); + } + user = defaultUser; + // Create sets and vectors for 'aquiring' subchunk temporary tables. + // Fill in _dbTblsAndSubchunks + DbTableSet dbTbls_; + IntVector subchunksVect_; + if (!_fragmentHasSubchunks) { + /// FUTURE: Why acquire anything if there are no subchunks in the fragment? + /// This branch never seems to happen, but this needs to be proven beyond any doubt. + for (auto const& scanTbl : scanInfo.infoTables) { + dbTbls_.emplace(scanTbl.db, scanTbl.table); + LOGS(_log, LOG_LVL_INFO, + "Task::Task scanTbl.db=" << scanTbl.db << " scanTbl.table=" << scanTbl.table); + } + LOGS(_log, LOG_LVL_INFO, + "fragment a db=" << _db << ":" << _chunkId << " dbTbls=" << util::printable(dbTbls_)); + } else { + for (TaskDbTbl const& fDbTbl : fragSubTables) { + /// Different subchunk fragments can require different tables. + /// FUTURE: It may save space to store these in UserQueryInfo as it seems + /// database and table names are consistent across chunks. + dbTbls_.emplace(fDbTbl.db, fDbTbl.tbl); + LOGS(_log, LOG_LVL_TRACE, + "Task::Task subchunk fDbTbl.db=" << fDbTbl.db << " fDbTbl.tbl=" << fDbTbl.tbl); + } + subchunksVect_ = fragSubchunkIds; + + LOGS(_log, LOG_LVL_DEBUG, + "fragment b db=" << _db << ":" << _chunkId << " dbTableSet" << util::printable(dbTbls_) + << " subChunks=" << util::printable(subchunksVect_)); + } + + _dbTblsAndSubchunks = make_unique(dbTbls_, subchunksVect_); +} + +Task::~Task() { _userQueryInfo.reset(); UserQueryInfo::uqMapErase(_qId); if (UserQueryInfo::uqMapGet(_qId) == nullptr) { @@ -267,6 +347,110 @@ vector Task::createTasks(shared_ptr const& taskMsg, return vect; } +std::vector Task::createTasksForChunk( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, proto::ScanInfo const& scanInfo, + bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, + std::shared_ptr const& sqlConnMgr, + std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { + QueryId qId = ujData->getQueryId(); + UberJobId ujId = ujData->getUberJobId(); + + UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + + string funcN(__func__); + funcN += " QID=" + to_string(qId) + " "; + + vector vect; + for (auto const& job : jsJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBody rbJobDesc(jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + auto const jdCzarId = rbJobDesc.required("czarId"); + auto const jdQueryId = rbJobDesc.required("queryId"); + if (jdQueryId != qId) { + throw TaskException(ERR_LOC, string("ujId=") + to_string(ujId) + " qId=" + to_string(qId) + + " QueryId mismatch Job qId=" + to_string(jdQueryId)); + } + auto const jdJobId = rbJobDesc.required("jobId"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_TRACE, + funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); + + auto const jdQueryFragments = rbJobDesc.required("queryFragments"); + int fragmentNumber = 0; + for (auto const& frag : jdQueryFragments) { + vector fragSubQueries; + vector fragSubchunkIds; + vector fragSubTables; + LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); + http::RequestBody rbFrag(frag); + auto const& jsQueries = rbFrag.required("queries"); + // TODO:UJ move to uberjob???, these should be the same for all jobs + for (auto const& subQ : jsQueries) { + http::RequestBody rbSubQ(subQ); + auto const subQuery = rbSubQ.required("subQuery"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); + fragSubQueries.push_back(subQuery); + } + auto const& resultTable = rbFrag.required("resultTable"); + auto const& jsSubIds = rbFrag.required("subchunkIds"); + for (auto const& scId : jsSubIds) { + fragSubchunkIds.push_back(scId); + } + auto const& jsSubTables = rbFrag.required("subchunkTables"); + + for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? + http::RequestBody rbScDbTable(scDbTable); + string scDb = rbScDbTable.required("scDb"); + string scTable = rbScDbTable.required("scTable"); + TaskDbTbl scDbTbl(scDb, scTable); + fragSubTables.push_back(scDbTbl); + } + + for (string const& fragSubQ : fragSubQueries) { + size_t templateId = userQueryInfo->addTemplate(fragSubQ); + if (fragSubchunkIds.empty()) { + bool const noSubchunks = false; + int const subchunkId = -1; + auto task = Task::Ptr(new Task( + ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, userQueryInfo, + templateId, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, + maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); + vect.push_back(task); + } else { + for (auto subchunkId : fragSubchunkIds) { + bool const hasSubchunks = true; + auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, + fragmentNumber, userQueryInfo, templateId, + hasSubchunks, subchunkId, jdQuerySpecDb, scanInfo, + scanInteractive, maxTableSizeMb, fragSubTables, + fragSubchunkIds, sendChannel, resultsHttpPort)); + vect.push_back(task); + } + } + } + ++fragmentNumber; + } + } + + for (auto taskPtr : vect) { + // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. + taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, + sqlConnMgr, queriesAndChunks)); + } + return vect; +} + void Task::action(util::CmdData* data) { string tIdStr = getIdStr(); if (_queryStarted.exchange(true)) { diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 22663ba5aa..c8b02f65b2 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -70,8 +70,14 @@ class QueryStatistics; namespace lsst::qserv::wbase { +class UberJobData; class UserQueryInfo; +class TaskException : public util::Issue { +public: + explicit TaskException(util::Issue::Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} +}; + /// Base class for tracking a database query for a worker Task. class TaskQueryRunner { public: @@ -81,6 +87,15 @@ class TaskQueryRunner { virtual void cancel() = 0; ///< Repeated calls to cancel() must be harmless. }; +/// Class for storing database + table name. +class TaskDbTbl { +public: + TaskDbTbl() = delete; + TaskDbTbl(std::string const& db_, std::string const& tbl_) : db(db_), tbl(tbl_) {} + std::string const db; + std::string const tbl; +}; + class Task; /// Base class for scheduling Tasks. @@ -103,7 +118,7 @@ class TaskScheduler { /// failure and should probably be removed when it is no longer needed. /// It depends on code in BlendScheduler to work. If the decision is made to keep it /// forever, dependency on BlendScheduler needs to be re-worked. -struct IdSet { +struct IdSet { // TODO:UJ delete if possible void add(std::string const& id) { std::lock_guard lock(mx); _ids.insert(id); @@ -158,6 +173,19 @@ class Task : public util::CommandForThreadPool { Task(TaskMsgPtr const& t, int fragmentNumber, std::shared_ptr const& userQueryInfo, size_t templateId, int subchunkId, std::shared_ptr const& sc, uint16_t resultsHttpPort = 8080); + // TODO:UJ too many parameters. + // - fragmentNumber seems pointless + // - hasSubchunks seems redundant. + // Hopefully, many are the same for all tasks and can be moved to ujData and userQueryInfo. + // Candidates: scanInfo, maxTableSizeMb, FileChannelShared, resultsHttpPort. + // Unfortunately, this will be much easier if it is done after xrootd method is removed. + Task(std::shared_ptr const& ujData, int jobId, int attemptCount, int chunkId, + int fragmentNumber, std::shared_ptr const& userQueryInfo, size_t templateId, + bool hasSubchunks, int subchunkId, std::string const& db, proto::ScanInfo const& scanInfo, + bool scanInteractive, int maxTableSizeMb, std::vector const& fragSubTables, + std::vector const& fragSubchunkIds, std::shared_ptr const& sc, + uint16_t resultsHttpPort = 8080); + Task& operator=(const Task&) = delete; Task(const Task&) = delete; virtual ~Task(); @@ -171,6 +199,16 @@ class Task : public util::CommandForThreadPool { std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort = 8080); + /// Read json to generate a vector of one or more task for a chunk. + static std::vector createTasksForChunk( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, proto::ScanInfo const& scanInfo, + bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr, + mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, + std::shared_ptr const& queriesAndChunks, + uint16_t resultsHttpPort = 8080); + void setQueryStatistics(std::shared_ptr const& qC); std::shared_ptr getSendChannel() const { return _sendChannel; } @@ -205,7 +243,6 @@ class Task : public util::CommandForThreadPool { TaskState state() const { return _state; } std::string getQueryString() const; - int getQueryFragmentNum() { return _queryFragmentNum; } std::string const& resultFileAbsPath() const { return _resultFileAbsPath; } std::string const& resultFileHttpUrl() const { return _resultFileHttpUrl; } bool setTaskQueryRunner( @@ -316,7 +353,6 @@ class Task : public util::CommandForThreadPool { int const _attemptCount = 0; ///< attemptCount from czar int const _queryFragmentNum; ///< The fragment number of the query in the task message. bool const _fragmentHasSubchunks; ///< True if the fragment in this query has subchunks. - bool const _hasDb; ///< true if db was in message from czar. std::string _db; ///< Task database int const _czarId; ///< czar Id from the task message. diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc new file mode 100644 index 0000000000..598727a372 --- /dev/null +++ b/src/wbase/UberJobData.cc @@ -0,0 +1,180 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "wbase/UberJobData.h" + +// System headers + +// Third party headers + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "http/Client.h" +#include "http/Exceptions.h" +#include "http/MetaModule.h" +#include "http/Method.h" +#include "http/RequestBody.h" +#include "http/RequestQuery.h" +#include "util/Bug.h" +#include "util/MultiError.h" +#include "wcontrol/Foreman.h" +#include "wpublish/ChunkInventory.h" +#include "wpublish/QueriesAndChunks.h" + +using namespace std; +using namespace nlohmann; + +namespace { + +LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.UberJobData"); + +} // namespace + +namespace lsst::qserv::wbase { + +UberJobData::UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, + std::string czarHost, int czarPort, uint64_t queryId, std::string const& workerId, + std::shared_ptr const& foreman, std::string const& authKey) + : _uberJobId(uberJobId), + _czarName(czarName), + _czarId(czarId), + _czarHost(czarHost), + _czarPort(czarPort), + _queryId(queryId), + _workerId(workerId), + _authKey(authKey), + _foreman(foreman), + _idStr(string("QID=") + to_string(_queryId) + ":ujId=" + to_string(_uberJobId)) {} + +void UberJobData::setFileChannelShared(std::shared_ptr const& fileChannelShared) { + if (_fileChannelShared != nullptr && _fileChannelShared != fileChannelShared) { + throw util::Bug(ERR_LOC, string(__func__) + " Trying to change _fileChannelShared"); + } + _fileChannelShared = fileChannelShared; +} + +void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, + uint64_t headerCount) { + string const funcN = cName(__func__); + LOGS(_log, LOG_LVL_TRACE, + funcN << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize + << " headerCount=" << headerCount); + + json request = {{"version", http::MetaModule::version}, + {"workerid", _foreman->chunkInventory()->id()}, + {"auth_key", _authKey}, + {"czar", _czarName}, + {"czarid", _czarId}, + {"queryid", _queryId}, + {"uberjobid", _uberJobId}, + {"fileUrl", httpFileUrl}, + {"rowCount", rowCount}, + {"fileSize", fileSize}, + {"headerCount", headerCount}}; + + auto const method = http::Method::POST; + vector const headers = {"Content-Type: application/json"}; + string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-ready"; + string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + http::Client client(method, url, request.dump(), headers); + + int maxTries = 2; // TODO:UJ set from config + bool transmitSuccess = false; + for (int j = 0; (!transmitSuccess && j < maxTries); ++j) { + try { + json const response = client.readAsJson(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, funcN << "Transmit success == 0"); + j = maxTries; /// There's no point in resending as the czar got the message and didn't like + /// it. + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " failed, ex: " + ex.what()); + } + } + + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, + funcN << "TODO:UJ NEED CODE Let czar find out through polling worker status??? Just throw the " + "result away???"); + } +} + +bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& task, + bool cancelled) { + string const funcN = cName(__func__); + LOGS(_log, LOG_LVL_INFO, funcN); + string errorMsg; + int errorCode = 0; + if (!multiErr.empty()) { + errorMsg = multiErr.toOneLineString(); + errorCode = multiErr.firstErrorCode(); + } else if (cancelled) { + errorMsg = "cancelled"; + errorCode = -1; + } + if (!errorMsg.empty() or (errorCode != 0)) { + errorMsg = + funcN + " error(s) in result for chunk #" + to_string(task->getChunkId()) + ": " + errorMsg; + LOGS(_log, LOG_LVL_ERROR, errorMsg); + } + + json request = {{"version", http::MetaModule::version}, + {"workerid", _foreman->chunkInventory()->id()}, + {"auth_key", _authKey}, + {"czar", _czarName}, + {"czarid", _czarId}, + {"queryid", _queryId}, + {"uberjobid", _uberJobId}, + {"errorCode", errorCode}, + {"errorMsg", errorMsg}}; + + auto const method = http::Method::POST; + vector const headers = {"Content-Type: application/json"}; + string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-error"; + string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + http::Client client(method, url, request.dump(), headers); + + int maxTries = 2; // TODO:UJ set from config + bool transmitSuccess = false; + for (int j = 0; !transmitSuccess && j < maxTries; ++j) { + try { + json const response = client.readAsJson(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, funcN << " transmit success == 0"); + j = maxTries; /// There's no point in resending as the czar got the message and didn't like + /// it. + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " failed, ex: " + ex.what()); + } + } + return transmitSuccess; +} + +} // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h new file mode 100644 index 0000000000..f4ab4e3030 --- /dev/null +++ b/src/wbase/UberJobData.h @@ -0,0 +1,121 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +#ifndef LSST_QSERV_WBASE_UBERJOBDATA_H +#define LSST_QSERV_WBASE_UBERJOBDATA_H + +// System headers +#include +#include +#include +#include +#include + +// Third-party headers +#include "nlohmann/json.hpp" + +// Qserv headers +#include "global/intTypes.h" +#include "qmeta/types.h" +#include "wbase/SendChannel.h" + +namespace lsst::qserv { + +namespace util { +class MultiError; +} + +namespace wcontrol { +class Foreman; +} +} // namespace lsst::qserv + +namespace lsst::qserv::wbase { + +class FileChannelShared; +class Task; + +/// This class tracks all Tasks associates with the UberJob on the worker +/// and reports status to the czar. +class UberJobData { +public: + using Ptr = std::shared_ptr; + + UberJobData() = delete; + UberJobData(UberJobData const&) = delete; + + static Ptr create(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, + std::string const& czarHost, int czarPort, uint64_t queryId, + std::string const& workerId, std::shared_ptr const& foreman, + std::string const& authKey) { + return Ptr(new UberJobData(uberJobId, czarName, czarId, czarHost, czarPort, queryId, workerId, + foreman, authKey)); + } + /// Set file channel for this UberJob + void setFileChannelShared(std::shared_ptr const& fileChannelShared); + + UberJobId getUberJobId() const { return _uberJobId; } + qmeta::CzarId getCzarId() const { return _czarId; } + std::string getCzarHost() const { return _czarHost; } + int getCzarPort() const { return _czarPort; } + uint64_t getQueryId() const { return _queryId; } + std::string getWorkerId() const { return _workerId; } + + /// Add the tasks defined in the UberJob to this UberJobData object. + void addTasks(std::vector> const& tasks) { + _ujTasks.insert(_ujTasks.end(), tasks.begin(), tasks.end()); + } + + /// Let the czar know the result is ready. + void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, + uint64_t headerCount); // TODO:UJ remove headerCount + + /// Let the Czar know there's been a problem. + bool responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); + + std::string getIdStr() const { return _idStr; } + std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } + +private: + UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, + int czarPort, uint64_t queryId, std::string const& workerId, + std::shared_ptr const& foreman, std::string const& authKey); + + UberJobId const _uberJobId; + std::string const _czarName; + qmeta::CzarId const _czarId; + std::string const _czarHost; + int const _czarPort; + QueryId const _queryId; + std::string const _workerId; + std::string const _authKey; + + std::shared_ptr const _foreman; + + std::vector> _ujTasks; + std::shared_ptr _fileChannelShared; + + std::string const _idStr; +}; + +} // namespace lsst::qserv::wbase + +#endif // LSST_QSERV_WBASE_UBERJOBDATA_H diff --git a/src/wbase/UserQueryInfo.cc b/src/wbase/UserQueryInfo.cc index 846be63fc4..79c24f07ed 100644 --- a/src/wbase/UserQueryInfo.cc +++ b/src/wbase/UserQueryInfo.cc @@ -24,6 +24,7 @@ // Qserv headers #include "util/Bug.h" +#include "wbase/UberJobData.h" // LSST headers #include "lsst/log/Log.h" @@ -101,4 +102,10 @@ std::string UserQueryInfo::getTemplate(size_t id) { return _templates[id]; } +void UserQueryInfo::addUberJob(std::shared_ptr const& ujData) { + lock_guard lockUq(_uberJobMapMtx); + UberJobId ujId = ujData->getUberJobId(); + _uberJobMap[ujId] = ujData; +} + } // namespace lsst::qserv::wbase diff --git a/src/wbase/UserQueryInfo.h b/src/wbase/UserQueryInfo.h index 27a7bb490a..4b7a799f03 100644 --- a/src/wbase/UserQueryInfo.h +++ b/src/wbase/UserQueryInfo.h @@ -35,6 +35,8 @@ // This header declarations namespace lsst::qserv::wbase { +class UberJobData; + /// This class contains information about a user query that is effectively the same /// for all Task's in the user query. class UserQueryInfo { @@ -63,6 +65,9 @@ class UserQueryInfo { /// @throws Bug if id is out of range. std::string getTemplate(size_t id); + /// Add an UberJobData object to the UserQueryInfo. + void addUberJob(std::shared_ptr const& ujData); + private: static Map _uqMap; static std::mutex _uqMapMtx; ///< protects _uqMap @@ -74,6 +79,10 @@ class UserQueryInfo { /// to alter existing indexes into the vector. std::vector _templates; std::mutex _uqMtx; ///< protects _templates; + + /// Map of all UberJobData objects on this worker for this User Query. + std::map> _uberJobMap; + std::mutex _uberJobMapMtx; ///< protects _uberJobMap; }; } // namespace lsst::qserv::wbase diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index b872e34d07..ff1a19752d 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -37,21 +37,18 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wsched.ChunkTasksQueue"); namespace lsst::qserv::wsched { -/// Queue a Task with other tasks on the same chunk. +/// Queue tasks from an uberjob. void ChunkTasksQueue::queueTask(std::vector const& tasks) { std::lock_guard lg(_mapMx); auto iter = _chunkMap.end(); + int prevChunkId = -1; // invalid chunkId number for (auto const& task : tasks) { int chunkId = task->getChunkId(); - if (iter != _chunkMap.end() && iter->first != chunkId) { - LOGS(_log, LOG_LVL_ERROR, - "All tasks grouped together must be on the same chunk." << " chunkA=" << iter->first - << " chunkB=" << chunkId); - throw util::Bug(ERR_LOC, "ChunkTasksQueue::queueTask mismatched chunkIds"); - } - /// If it's the first time through, or the chunkId is different than the previous one, then - /// find the correct ChunkTask. - if (iter == _chunkMap.end() || iter->first != chunkId) { + // If it's the first time through, or the chunkId is different than the previous one, then + // find the correct ChunkTask. UberJobs are constructed in a way that makes it likely + // that subchunks for the same chunk will be grouped together in `tasks`. + if (iter == _chunkMap.end() || prevChunkId != chunkId) { + prevChunkId = chunkId; iter = _chunkMap.find(chunkId); if (iter == _chunkMap.end()) { // Correct ChunkTask wasn't found, make a new one. diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index ce01f5f0d4..8695f78bb2 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -182,28 +182,30 @@ void ScanScheduler::queCmd(vector const& cmds) { int jid = 0; // Convert to a vector of tasks for (auto const& cmd : cmds) { - wbase::Task::Ptr t = dynamic_pointer_cast(cmd); - if (t == nullptr) { + wbase::Task::Ptr tsk = dynamic_pointer_cast(cmd); + if (tsk == nullptr) { throw util::Bug(ERR_LOC, getName() + " queCmd could not be converted to Task or was nullptr"); } if (first) { first = false; - qid = t->getQueryId(); - jid = t->getJobId(); + qid = tsk->getQueryId(); + jid = tsk->getJobId(); QSERV_LOGCONTEXT_QUERY_JOB(qid, jid); } else { - if (qid != t->getQueryId() || jid != t->getJobId()) { - LOGS(_log, LOG_LVL_ERROR, - " mismatch multiple query/job ids in single queCmd " - << " expected QID=" << qid << " got=" << t->getQueryId() - << " expected JID=" << jid << " got=" << t->getJobId()); + if (qid != tsk->getQueryId()) { + string eMsg("Mismatch multiple query/job ids in single queCmd "); + eMsg += " expected QID=" + to_string(qid) + " got=" + to_string(tsk->getQueryId()); + eMsg += " expected JID=" + to_string(qid) + " got=" + to_string(tsk->getJobId()); + LOGS(_log, LOG_LVL_ERROR, eMsg); // This could cause difficult to detect problems later on. - throw util::Bug(ERR_LOC, "Mismatch multiple query/job ids in single queCmd"); + throw util::Bug(ERR_LOC, eMsg); return; } } - tasks.push_back(t); - LOGS(_log, LOG_LVL_INFO, getName() << " queCmd " << t->getIdStr()); + + tsk->setMemMan(_memMan); + tasks.push_back(tsk); + LOGS(_log, LOG_LVL_INFO, getName() << " queCmd " << tsk->getIdStr()); } // Queue the tasks { diff --git a/src/xrdsvc/CMakeLists.txt b/src/xrdsvc/CMakeLists.txt index 9fc8d622d1..2e5455bab1 100644 --- a/src/xrdsvc/CMakeLists.txt +++ b/src/xrdsvc/CMakeLists.txt @@ -6,6 +6,7 @@ target_sources(xrdsvc PRIVATE HttpModule.cc HttpMonitorModule.cc HttpReplicaMgtModule.cc + HttpWorkerCzarModule.cc HttpSvc.cc SsiProvider.cc SsiRequest.cc diff --git a/src/xrdsvc/HttpReplicaMgtModule.cc b/src/xrdsvc/HttpReplicaMgtModule.cc index afa81d74c8..14fdde32af 100644 --- a/src/xrdsvc/HttpReplicaMgtModule.cc +++ b/src/xrdsvc/HttpReplicaMgtModule.cc @@ -28,6 +28,7 @@ #include // Third party headers +#include "lsst/log/Log.h" #include "XrdSsi/XrdSsiCluster.hh" // Qserv headers @@ -48,6 +49,10 @@ extern XrdSsiProvider* XrdSsiProviderLookup; using namespace std; using json = nlohmann::json; +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.HttpReplicaMgt"); +} + namespace { // These markers if reported in the extended error response object of the failed // requests could be used by a caller for refining the completion status @@ -80,7 +85,6 @@ HttpReplicaMgtModule::HttpReplicaMgtModule(string const& context, json HttpReplicaMgtModule::executeImpl(string const& subModuleName) { string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; - debug(func); enforceInstanceId(func, wconfig::WorkerConfig::instance()->replicationInstanceId()); enforceWorkerId(func); if (subModuleName == "GET") diff --git a/src/xrdsvc/HttpSvc.cc b/src/xrdsvc/HttpSvc.cc index f30c82a0c1..49781fc24e 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/xrdsvc/HttpSvc.cc @@ -33,6 +33,7 @@ #include "wpublish/ChunkInventory.h" #include "xrdsvc/HttpMonitorModule.h" #include "xrdsvc/HttpReplicaMgtModule.h" +#include "xrdsvc/HttpWorkerCzarModule.h" // LSST headers #include "lsst/log/Log.h" @@ -134,6 +135,12 @@ uint16_t HttpSvc::start() { HttpReplicaMgtModule::process(::serviceName, self->_foreman, req, resp, "REBUILD", http::AuthType::REQUIRED); }}}); + _httpServerPtr->addHandlers( + {{"POST", "/queryjob", + [self](shared_ptr const& req, shared_ptr const& resp) { + HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "QUERYJOB", + http::AuthType::REQUIRED); + }}}); _httpServerPtr->start(); // Initialize the I/O context and start the service threads. At this point diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc new file mode 100644 index 0000000000..8c0748b5ae --- /dev/null +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -0,0 +1,213 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "xrdsvc/HttpWorkerCzarModule.h" + +// System headers +#include +#include +#include + +// Third party headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "http/Client.h" // TODO:UJ will probably need to be removed +#include "http/Exceptions.h" +#include "http/MetaModule.h" +#include "http/RequestBody.h" +#include "http/RequestQuery.h" +#include "mysql/MySqlUtils.h" +#include "qmeta/types.h" +#include "util/String.h" +#include "util/Timer.h" +#include "wbase/FileChannelShared.h" +#include "wbase/Task.h" +#include "wbase/UberJobData.h" +#include "wbase/UserQueryInfo.h" +#include "wconfig/WorkerConfig.h" +#include "wcontrol/Foreman.h" +#include "wcontrol/ResourceMonitor.h" +#include "wpublish/ChunkInventory.h" +#include "xrdsvc/SsiProvider.h" +#include "xrdsvc/XrdName.h" + +using namespace std; +using json = nlohmann::json; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.HttpReplicaMgt"); +} + +namespace { +// These markers if reported in the extended error response object of the failed +// requests could be used by a caller for refining the completion status +// of the corresponding Controller-side operation. +// TODO:UJ Are these errors seem useful enought to be centralized ??? +json const extErrorInvalidParam = json::object({{"invalid_param", 1}}); +json const extErrorReplicaInUse = json::object({{"in_use", 1}}); + +} // namespace + +namespace lsst::qserv::xrdsvc { + +void HttpWorkerCzarModule::process(string const& context, shared_ptr const& foreman, + shared_ptr const& req, + shared_ptr const& resp, string const& subModuleName, + http::AuthType const authType) { + HttpWorkerCzarModule module(context, foreman, req, resp); + module.execute(subModuleName, authType); +} + +HttpWorkerCzarModule::HttpWorkerCzarModule(string const& context, + shared_ptr const& foreman, + shared_ptr const& req, + shared_ptr const& resp) + : HttpModule(context, foreman, req, resp) {} + +json HttpWorkerCzarModule::executeImpl(string const& subModuleName) { + string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; + enforceInstanceId(func, wconfig::WorkerConfig::instance()->replicationInstanceId()); + enforceWorkerId(func); + if (subModuleName == "QUERYJOB") return _queryJob(); + throw invalid_argument(context() + func + " unsupported sub-module"); +} + +json HttpWorkerCzarModule::_queryJob() { + debug(__func__); + checkApiVersion(__func__, 34); + // At this point, API version, correct worker, and auth have been checked. + json jsRet = _handleQueryJob(__func__); + return jsRet; +} + +json HttpWorkerCzarModule::_handleQueryJob(string const& func) { + json jsRet; + vector ujTasks; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + auto const& jsReq = body().objJson; + string const targetWorkerId = body().required("worker"); + + http::RequestBody rbCzar(body().required("czar")); + auto czarName = rbCzar.required("name"); + auto czarId = rbCzar.required("id"); + auto czarPort = rbCzar.required("management-port"); + auto czarHostName = rbCzar.required("management-host-name"); + LOGS(_log, LOG_LVL_TRACE, + __func__ << " czar n=" << czarName << " id=" << czarId << " p=" << czarPort + << " h=" << czarHostName); + + http::RequestBody rbUberJob(body().required("uberjob")); + auto ujQueryId = rbUberJob.required("queryid"); + auto ujId = rbUberJob.required("uberjobid"); + auto ujCzarId = rbUberJob.required("czarid"); + auto ujJobs = rbUberJob.required("jobs"); + LOGS(_log, LOG_LVL_TRACE, + __func__ << " uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); + + auto ujData = wbase::UberJobData::create(ujId, czarName, czarId, czarHostName, czarPort, ujQueryId, + targetWorkerId, foreman(), authKey()); + + // Find the entry for this queryId, creat a new one if needed. + wbase::UserQueryInfo::Ptr userQueryInfo = wbase::UserQueryInfo::uqMapInsert(ujQueryId); + userQueryInfo->addUberJob(ujData); + + auto channelShared = + wbase::FileChannelShared::create(ujData, czarId, czarHostName, czarPort, targetWorkerId); + ujData->setFileChannelShared(channelShared); + + // TODO:UJ These items should be stored higher in the message structure as they get + // duplicated and should always be the same within an UberJob. + QueryId jdQueryId = 0; + proto::ScanInfo scanInfo; + bool scanInfoSet = false; + bool jdScanInteractive = false; + int jdMaxTableSize = 0; + + for (auto const& job : ujJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBody rbJobDesc(jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + auto const jdCzarId = rbJobDesc.required("czarId"); + jdQueryId = rbJobDesc.required("queryId"); + auto const jdJobId = rbJobDesc.required("jobId"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + jdScanInteractive = rbJobDesc.required("scanInteractive"); + jdMaxTableSize = rbJobDesc.required("maxTableSize"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_TRACE, + __func__ << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSize << " chunkId=" << jdChunkId); + + auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); + if (!scanInfoSet) { + for (auto const& tbl : jdChunkScanTables) { + http::RequestBody rbTbl(tbl); + auto const& chunkScanDb = rbTbl.required("db"); + auto const& lockInMemory = rbTbl.required("lockInMemory"); + auto const& chunkScanTable = rbTbl.required("table"); + auto const& tblScanRating = rbTbl.required("tblScanRating"); + LOGS(_log, LOG_LVL_TRACE, + __func__ << " chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory + << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); + scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, + tblScanRating); + scanInfoSet = true; + } + } + scanInfo.scanRating = jdScanPriority; + } + + // create tasks and add them to ujData + auto chunkTasks = wbase::Task::createTasksForChunk( + ujData, ujJobs, channelShared, scanInfo, jdScanInteractive, jdMaxTableSize, + foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), foreman()->sqlConnMgr(), + foreman()->queriesAndChunks(), foreman()->httpPort()); + ujTasks.insert(ujTasks.end(), chunkTasks.begin(), chunkTasks.end()); + + channelShared->setTaskCount(ujTasks.size()); + ujData->addTasks(ujTasks); + + util::Timer timer; + timer.start(); + foreman()->processTasks(ujTasks); // Queues tasks to be run later. + timer.stop(); + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); + + string note = string("qId=") + to_string(ujQueryId) + " ujId=" + to_string(ujId) + + " tasks in uberJob=" + to_string(channelShared->getTaskCount()); + jsRet = {{"success", 1}, {"errortype", "none"}, {"note", note}}; + + } catch (wbase::TaskException const& texp) { + LOGS(_log, LOG_LVL_ERROR, "wbase::TaskException received " << texp.what()); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", texp.what()}}; + } + return jsRet; +} + +} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/xrdsvc/HttpWorkerCzarModule.h new file mode 100644 index 0000000000..bb75a63c55 --- /dev/null +++ b/src/xrdsvc/HttpWorkerCzarModule.h @@ -0,0 +1,88 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H +#define LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H + +// System headers +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// Qserv headers +#include "qmeta/types.h" +#include "xrdsvc/HttpModule.h" + +namespace lsst::qserv::qhttp { +class Request; +class Response; +} // namespace lsst::qserv::qhttp + +namespace lsst::qserv::wcontrol { +class Foreman; +} // namespace lsst::qserv::wcontrol + +namespace lsst::qserv::xrdsvc { +class SsiProviderServer; +} // namespace lsst::qserv::xrdsvc + +// This header declarations +namespace lsst::qserv::xrdsvc { + +/// This class handles Http message from the czar to the worker. +class HttpWorkerCzarModule : public xrdsvc::HttpModule { +public: + /// @note supported values for parameter 'subModuleName' are: + /// 'QUERYJOB' - Convert an UberJob message into Tasks and a send channel. + /// @throws std::invalid_argument for unknown values of parameter 'subModuleName' + static void process(std::string const& context, std::shared_ptr const& foreman, + std::shared_ptr const& req, + std::shared_ptr const& resp, std::string const& subModuleName, + http::AuthType const authType = http::AuthType::NONE); + + HttpWorkerCzarModule() = delete; + HttpWorkerCzarModule(HttpWorkerCzarModule const&) = delete; + HttpWorkerCzarModule& operator=(HttpWorkerCzarModule const&) = delete; + + ~HttpWorkerCzarModule() final = default; + +protected: + virtual nlohmann::json executeImpl(std::string const& subModuleName) final; + +private: + HttpWorkerCzarModule(std::string const& context, std::shared_ptr const& foreman, + std::shared_ptr const& req, + std::shared_ptr const& resp); + + /// Handle an UberJob message from the czar to run it on this worker by calling _handleQueryJob. + nlohmann::json _queryJob(); + + /// Handle an UberJob message from the czar to run it on this worker, this does + /// work of deciphering the message, creating UberJobData objects and Task objects. + nlohmann::json _handleQueryJob(std::string const& func); +}; + +} // namespace lsst::qserv::xrdsvc + +#endif // LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H From 1c92a9276296cd2a050bb9e9ee003368a0c9b345 Mon Sep 17 00:00:00 2001 From: John Gates Date: Mon, 22 Jul 2024 10:19:16 -0700 Subject: [PATCH 02/15] CzarFamilyMap create now waits for a successful read. --- src/CMakeLists.txt | 4 +- .../templates/http/etc/qserv-czar.cnf.jinja | 2 - .../templates/proxy/etc/qserv-czar.cnf.jinja | 10 +- src/cconfig/CzarConfig.cc | 7 +- src/cconfig/CzarConfig.h | 58 +- src/ccontrol/CMakeLists.txt | 2 - src/ccontrol/MergingHandler.cc | 25 +- src/ccontrol/MergingHandler.h | 5 - src/ccontrol/UserQueryAsyncResult.cc | 3 + src/ccontrol/UserQueryFactory.cc | 6 +- src/ccontrol/UserQueryFactory.h | 4 +- src/ccontrol/UserQuerySelect.cc | 256 +++++---- src/ccontrol/UserQuerySelect.h | 10 +- src/czar/ActiveWorker.cc | 311 +++++++++++ src/czar/ActiveWorker.h | 251 +++++++++ src/czar/CMakeLists.txt | 3 +- src/czar/Czar.cc | 124 +++-- src/czar/Czar.h | 68 ++- src/czar/CzarChunkMap.cc | 146 +++-- src/czar/CzarChunkMap.h | 33 +- src/czar/CzarRegistry.cc | 88 ++- src/czar/CzarRegistry.h | 74 ++- src/czar/HttpCzarWorkerModule.cc | 61 ++- src/czar/HttpCzarWorkerModule.h | 10 +- src/czar/HttpSvc.cc | 5 + src/global/CMakeLists.txt | 1 - src/global/ResourceUnit.cc | 127 ----- src/global/ResourceUnit.h | 31 +- src/global/clock_defs.h | 4 + src/global/intTypes.h | 3 +- src/global/testResourceUnit.cc | 91 ---- src/http/BaseModule.h | 2 + src/http/Client.cc | 1 - src/http/Module.h | 3 - src/http/RequestBodyJSON.h | 18 +- src/proto/CMakeLists.txt | 16 - src/proto/FakeProtocolFixture.h | 92 ---- src/proto/ProtoImporter.h | 63 --- src/proto/testProtocol.cc | 183 ------- src/proto/worker.proto | 78 --- src/protojson/CMakeLists.txt | 40 ++ src/{proto => protojson}/ScanTableInfo.cc | 50 +- src/{proto => protojson}/ScanTableInfo.h | 42 +- src/protojson/UberJobMsg.cc | 511 +++++++++++++++++ src/protojson/UberJobMsg.h | 311 +++++++++++ src/protojson/WorkerQueryStatusData.cc | 512 ++++++++++++++++++ src/protojson/WorkerQueryStatusData.h | 478 ++++++++++++++++ src/protojson/testStatusData.cc | 181 +++++++ src/protojson/testUberJobMsg.cc | 130 +++++ src/qana/CMakeLists.txt | 1 - src/qana/QueryMapping.h | 2 + src/qana/ScanTablePlugin.cc | 18 +- src/qana/ScanTablePlugin.h | 6 +- src/qdisp/CMakeLists.txt | 7 +- src/qdisp/CzarStats.cc | 12 +- src/qdisp/CzarStats.h | 16 +- src/qdisp/Executive.cc | 225 ++++---- src/qdisp/Executive.h | 130 ++--- src/qdisp/JobBase.cc | 54 -- src/qdisp/JobBase.h | 78 --- src/qdisp/JobDescription.cc | 45 +- src/qdisp/JobDescription.h | 31 +- src/qdisp/JobQuery.cc | 126 +---- src/qdisp/JobQuery.h | 107 ++-- src/qdisp/ResponseHandler.h | 16 +- src/qdisp/SharedResources.h | 60 -- src/qdisp/UberJob.cc | 206 ++++--- src/qdisp/UberJob.h | 70 +-- src/qdisp/XrdSsiMocks.cc | 312 ----------- src/qdisp/XrdSsiMocks.h | 72 --- src/qdisp/testQDisp.cc | 319 ++++++----- src/qhttp/Server.cc | 24 - src/qmeta/QMetaMysql.cc | 9 +- src/qmeta/types.h | 2 +- src/qproc/CMakeLists.txt | 2 - src/qproc/ChunkQuerySpec.h | 11 +- src/qproc/ChunkSpec.cc | 10 +- src/qproc/ChunkSpec.h | 2 + src/qproc/QuerySession.cc | 7 +- src/qproc/QuerySession.h | 2 + src/qproc/TaskMsgFactory.cc | 308 ----------- src/qproc/TaskMsgFactory.h | 93 ---- src/qproc/testQueryAnaGeneral.cc | 6 +- src/query/CMakeLists.txt | 1 - src/query/QueryContext.h | 4 +- src/query/QueryTemplate.cc | 16 + src/query/QueryTemplate.h | 2 + src/rproc/InfileMerger.cc | 22 +- src/util/CMakeLists.txt | 2 +- src/util/ConfigValMap.h | 2 + src/util/InstanceCount.cc | 4 +- src/util/Mutex.cc | 2 +- src/util/Mutex.h | 33 +- src/{qdisp => util}/QdispPool.cc | 13 +- src/{qdisp => util}/QdispPool.h | 22 +- src/util/testMutex.cc | 58 +- src/util/xrootd.cc | 89 --- src/util/xrootd.h | 42 -- src/wbase/CMakeLists.txt | 1 - src/wbase/FileChannelShared.cc | 211 +++----- src/wbase/FileChannelShared.h | 24 +- src/wbase/MsgProcessor.h | 64 --- src/wbase/SendChannel.cc | 80 +-- src/wbase/SendChannel.h | 49 +- src/wbase/Task.cc | 443 ++++++++------- src/wbase/Task.h | 123 ++--- src/wbase/UberJobData.cc | 196 +++++-- src/wbase/UberJobData.h | 105 +++- src/wbase/UserQueryInfo.cc | 87 +-- src/wbase/UserQueryInfo.h | 51 +- src/wbase/WorkerCommand.cc | 49 -- src/wbase/WorkerCommand.h | 96 ---- src/wconfig/WorkerConfig.h | 31 ++ src/wcontrol/CMakeLists.txt | 1 + src/wcontrol/Foreman.cc | 45 +- src/wcontrol/Foreman.h | 61 ++- src/wcontrol/WCzarInfoMap.cc | 186 +++++++ src/wcontrol/WCzarInfoMap.h | 129 +++++ src/wcontrol/WorkerStats.cc | 6 +- src/wcontrol/WorkerStats.h | 2 +- src/wdb/CMakeLists.txt | 4 +- src/wdb/ChunkResource.cc | 21 - src/wdb/QueryRunner.cc | 3 +- src/wdb/QueryRunner.h | 3 - src/wdb/QuerySql.cc | 133 ----- src/wdb/QuerySql.h | 72 --- src/wdb/QuerySql_Batch.h | 82 --- src/wdb/testQueryRunner.cc | 119 +++- src/wdb/testQuerySql.cc | 99 ---- src/wpublish/ChunkInventory.cc | 20 - src/wpublish/QueriesAndChunks.cc | 50 +- src/wpublish/QueriesAndChunks.h | 18 +- src/wpublish/QueryStatistics.cc | 5 +- src/wpublish/QueryStatistics.h | 15 +- src/wsched/BlendScheduler.cc | 5 +- src/wsched/ChunkTasksQueue.h | 2 +- src/wsched/GroupScheduler.cc | 2 +- src/wsched/testSchedulers.cc | 39 +- src/xrdreq/CMakeLists.txt | 49 -- src/xrdreq/QservRequest.cc | 216 -------- src/xrdreq/QservRequest.h | 120 ---- src/xrdreq/QueryManagementAction.cc | 137 ----- src/xrdreq/QueryManagementAction.h | 96 ---- src/xrdreq/QueryManagementRequest.cc | 91 ---- src/xrdreq/QueryManagementRequest.h | 95 ---- src/xrdreq/qserv-query-management.cc | 154 ------ src/xrdsvc/CMakeLists.txt | 5 +- src/xrdsvc/ChannelStream.cc | 115 ---- src/xrdsvc/ChannelStream.h | 75 --- src/xrdsvc/HttpReplicaMgtModule.cc | 9 +- src/xrdsvc/HttpReplicaMgtModule.h | 4 - src/xrdsvc/HttpSvc.cc | 8 +- src/xrdsvc/HttpWorkerCzarModule.cc | 239 +++++--- src/xrdsvc/HttpWorkerCzarModule.h | 8 + src/xrdsvc/SsiProvider.cc | 51 +- src/xrdsvc/SsiRequest.cc | 389 ------------- src/xrdsvc/SsiRequest.h | 128 ----- src/xrdsvc/SsiService.cc | 270 --------- src/xrdsvc/StreamBuffer.cc | 139 ----- src/xrdsvc/StreamBuffer.h | 113 ---- 160 files changed, 5784 insertions(+), 6694 deletions(-) create mode 100644 src/czar/ActiveWorker.cc create mode 100644 src/czar/ActiveWorker.h delete mode 100644 src/global/testResourceUnit.cc delete mode 100644 src/proto/FakeProtocolFixture.h delete mode 100644 src/proto/ProtoImporter.h delete mode 100644 src/proto/testProtocol.cc create mode 100644 src/protojson/CMakeLists.txt rename src/{proto => protojson}/ScanTableInfo.cc (69%) rename src/{proto => protojson}/ScanTableInfo.h (73%) create mode 100644 src/protojson/UberJobMsg.cc create mode 100644 src/protojson/UberJobMsg.h create mode 100644 src/protojson/WorkerQueryStatusData.cc create mode 100644 src/protojson/WorkerQueryStatusData.h create mode 100644 src/protojson/testStatusData.cc create mode 100644 src/protojson/testUberJobMsg.cc delete mode 100644 src/qdisp/JobBase.cc delete mode 100644 src/qdisp/JobBase.h delete mode 100644 src/qdisp/SharedResources.h delete mode 100644 src/qdisp/XrdSsiMocks.cc delete mode 100644 src/qdisp/XrdSsiMocks.h delete mode 100644 src/qproc/TaskMsgFactory.cc delete mode 100644 src/qproc/TaskMsgFactory.h rename src/{qdisp => util}/QdispPool.cc (96%) rename src/{qdisp => util}/QdispPool.h (91%) delete mode 100644 src/util/xrootd.cc delete mode 100644 src/util/xrootd.h delete mode 100644 src/wbase/MsgProcessor.h delete mode 100644 src/wbase/WorkerCommand.cc delete mode 100644 src/wbase/WorkerCommand.h create mode 100644 src/wcontrol/WCzarInfoMap.cc create mode 100644 src/wcontrol/WCzarInfoMap.h delete mode 100644 src/wdb/QuerySql.cc delete mode 100644 src/wdb/QuerySql.h delete mode 100644 src/wdb/QuerySql_Batch.h delete mode 100644 src/wdb/testQuerySql.cc delete mode 100644 src/xrdreq/CMakeLists.txt delete mode 100644 src/xrdreq/QservRequest.cc delete mode 100644 src/xrdreq/QservRequest.h delete mode 100644 src/xrdreq/QueryManagementAction.cc delete mode 100644 src/xrdreq/QueryManagementAction.h delete mode 100644 src/xrdreq/QueryManagementRequest.cc delete mode 100644 src/xrdreq/QueryManagementRequest.h delete mode 100644 src/xrdreq/qserv-query-management.cc delete mode 100644 src/xrdsvc/ChannelStream.cc delete mode 100644 src/xrdsvc/ChannelStream.h delete mode 100644 src/xrdsvc/SsiRequest.cc delete mode 100644 src/xrdsvc/SsiRequest.h delete mode 100644 src/xrdsvc/SsiService.cc delete mode 100644 src/xrdsvc/StreamBuffer.cc delete mode 100644 src/xrdsvc/StreamBuffer.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 437776ac2c..822a8ae3b5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -67,6 +67,7 @@ add_subdirectory(mysql) add_subdirectory(parser) add_subdirectory(partition) add_subdirectory(proto) +add_subdirectory(protojson) add_subdirectory(proxy) add_subdirectory(qana) add_subdirectory(qdisp) @@ -86,5 +87,6 @@ add_subdirectory(wpublish) add_subdirectory(wsched) add_subdirectory(www) add_subdirectory(xrdlog) -add_subdirectory(xrdreq) add_subdirectory(xrdsvc) + + diff --git a/src/admin/templates/http/etc/qserv-czar.cnf.jinja b/src/admin/templates/http/etc/qserv-czar.cnf.jinja index 8b9847cbe4..e996142eef 100644 --- a/src/admin/templates/http/etc/qserv-czar.cnf.jinja +++ b/src/admin/templates/http/etc/qserv-czar.cnf.jinja @@ -108,8 +108,6 @@ largestPriority = 3 vectRunSizes = 50:50:50:50 # Minimum number of threads running for each queue. No spaces. Values separated by ':' vectMinRunningSizes = 0:1:3:3 -# Maximum number of QueryRequests allowed to be running at one time. -qReqPseudoFifoMaxRunning = 299 [replication] diff --git a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja index 2878854e12..49dc123c61 100644 --- a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja +++ b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja @@ -97,21 +97,19 @@ notifyWorkersOnCzarRestart = 1 #[debug] #chunkLimit = -1 -# Please see qdisp/QdispPool.h QdispPool::QdispPool for more information +# Please see util/QdispPool.h QdispPool::QdispPool for more information [qdisppool] #size of the pool -poolSize = 50 +poolSize = 1000 # Low numbers are higher priority. Largest priority 3 creates 4 priority queues 0, 1, 2, 3 # Must be greater than 0. largestPriority = 3 # Maximum number of threads running for each queue. No spaces. Values separated by ':' # Using largestPriority = 2 and vectRunsizes = 3:5:8 # queue 0 would have runSize 3, queue 1 would have runSize 5, and queue 2 would have runSize 8. -vectRunSizes = 50:50:50:50 +vectRunSizes = 800:800:500:500 # Minimum number of threads running for each queue. No spaces. Values separated by ':' -vectMinRunningSizes = 0:1:3:3 -# Maximum number of QueryRequests allowed to be running at one time. -qReqPseudoFifoMaxRunning = 299 +vectMinRunningSizes = 0:3:3:3 [replication] diff --git a/src/cconfig/CzarConfig.cc b/src/cconfig/CzarConfig.cc index c27e7336ca..bf2ede5639 100644 --- a/src/cconfig/CzarConfig.cc +++ b/src/cconfig/CzarConfig.cc @@ -63,10 +63,9 @@ namespace lsst::qserv::cconfig { std::mutex CzarConfig::_mtxOnInstance; -std::shared_ptr CzarConfig::_instance; +CzarConfig::Ptr CzarConfig::_instance; -std::shared_ptr CzarConfig::create(std::string const& configFileName, - std::string const& czarName) { +CzarConfig::Ptr CzarConfig::create(std::string const& configFileName, std::string const& czarName) { std::lock_guard const lock(_mtxOnInstance); if (_instance == nullptr) { _instance = std::shared_ptr(new CzarConfig(util::ConfigStore(configFileName), czarName)); @@ -74,7 +73,7 @@ std::shared_ptr CzarConfig::create(std::string const& configFileName return _instance; } -std::shared_ptr CzarConfig::instance() { +CzarConfig::Ptr CzarConfig::instance() { std::lock_guard const lock(_mtxOnInstance); if (_instance == nullptr) { throw std::logic_error("CzarConfig::" + std::string(__func__) + ": instance has not been created."); diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 201a89602f..40aab039b4 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -59,6 +59,7 @@ namespace lsst::qserv::cconfig { */ class CzarConfig { public: + using Ptr = std::shared_ptr; /** * Create an instance of CzarConfig and load parameters from the specifid file. * @note One has to call this method at least once before trying to obtain @@ -69,7 +70,7 @@ class CzarConfig { * @param czarName - the unique name of Czar. * @return the shared pointer to the configuration object */ - static std::shared_ptr create(std::string const& configFileName, std::string const& czarName); + static Ptr create(std::string const& configFileName, std::string const& czarName); /** * Get a pointer to an instance that was created by the last call to @@ -77,7 +78,7 @@ class CzarConfig { * @return the shared pointer to the configuration object * @throws std::logic_error when attempting to call the bethod before creating an instance. */ - static std::shared_ptr instance(); + static Ptr instance(); CzarConfig() = delete; CzarConfig(CzarConfig const&) = delete; @@ -196,6 +197,35 @@ class CzarConfig { /// the method then the monitoring will be disabled. unsigned int czarStatsUpdateIvalSec() const { return _czarStatsUpdateIvalSec->getVal(); } + /// @return The maximum retain period for keeping in memory the relevant metrics + /// captured by the Czar monitoring system. If 0 is returned by the method then + /// query history archiving will be disabled. + /// @note Setting the limit too high may be potentially result in runing onto + /// the OOM situation. + unsigned int czarStatsRetainPeriodSec() const { return _czarStatsRetainPeriodSec->getVal(); } + + /// A worker is considered fully ALIVE if the last update from the worker has been + /// heard in less than _activeWorkerTimeoutAliveSecs seconds. + int getActiveWorkerTimeoutAliveSecs() const { return _activeWorkerTimeoutAliveSecs->getVal(); } + + /// A worker is considered DEAD if it hasn't been heard from in more than + /// _activeWorkerTimeoutDeadSecs. + int getActiveWorkerTimeoutDeadSecs() const { return _activeWorkerTimeoutDeadSecs->getVal(); } + + /// Max lifetime of a message to be sent to an active worker. If the czar has been + /// trying to send a message to a worker and has failed for this many seconds, + /// it gives up at this point, removing elements of the message to save memory. + int getActiveWorkerMaxLifetimeSecs() const { return _activeWorkerMaxLifetimeSecs->getVal(); } + + /// The maximum number of chunks (basically Jobs) allowed in a single UberJob. + int getUberJobMaxChunks() const { return _uberJobMaxChunks->getVal(); } + + /// Return the maximum number of http connections to use for czar commands. + int getCommandMaxHttpConnections() const { return _commandMaxHttpConnections->getVal(); } + + /// Return the sleep time (in milliseconds) between messages sent to active workers. + int getMonitorSleepTimeMilliSec() const { return _monitorSleepTimeMilliSec->getVal(); } + // Parameters of the Czar management service std::string const& replicationInstanceId() const { return _replicationInstanceId->getVal(); } @@ -295,7 +325,7 @@ class CzarConfig { CVTIntPtr _maxSqlConnectionAttempts = util::ConfigValTInt::create(_configValMap, "resultdb", "maxsqlconnectionattempts", notReq, 10); CVTIntPtr _resultMaxHttpConnections = - util::ConfigValTInt::create(_configValMap, "resultdb", "maxhttpconnections", notReq, 8192); + util::ConfigValTInt::create(_configValMap, "resultdb", "maxhttpconnections", notReq, 2000); CVTIntPtr _oldestResultKeptDays = util::ConfigValTInt::create(_configValMap, "resultdb", "oldestResultKeptDays", notReq, 30); CVTIntPtr _oldestAsyncResultKeptSeconds = util::ConfigValTInt::create( @@ -351,9 +381,9 @@ class CzarConfig { CVTIntPtr _qdispMaxPriority = util::ConfigValTInt::create(_configValMap, "qdisppool", "largestPriority", notReq, 2); CVTStrPtr _qdispVectRunSizes = - util::ConfigValTStr::create(_configValMap, "qdisppool", "vectRunSizes", notReq, "50:50:50:50"); + util::ConfigValTStr::create(_configValMap, "qdisppool", "vectRunSizes", notReq, "800:800:500:50"); CVTStrPtr _qdispVectMinRunningSizes = - util::ConfigValTStr::create(_configValMap, "qdisppool", "vectMinRunningSizes", notReq, "0:1:3:3"); + util::ConfigValTStr::create(_configValMap, "qdisppool", "vectMinRunningSizes", notReq, "0:3:3:3"); CVTIntPtr _xrootdSpread = util::ConfigValTInt::create(_configValMap, "tuning", "xrootdSpread", notReq, 4); CVTIntPtr _qMetaSecsBetweenChunkCompletionUpdates = util::ConfigValTInt::create( @@ -395,6 +425,24 @@ class CzarConfig { CVTStrPtr _httpUser = util::ConfigValTStr::create(_configValMap, "http", "user", notReq, ""); CVTStrPtr _httpPassword = util::ConfigValTStr::create(_configValMap, "http", "password", notReq, "", hidden); + + // Active Worker + CVTIntPtr _activeWorkerTimeoutAliveSecs = // 5min + util::ConfigValTInt::create(_configValMap, "activeworker", "timeoutAliveSecs", notReq, 60 * 5); + CVTIntPtr _activeWorkerTimeoutDeadSecs = // 10min + util::ConfigValTInt::create(_configValMap, "activeworker", "timeoutDeadSecs", notReq, 60 * 10); + CVTIntPtr _activeWorkerMaxLifetimeSecs = // 1hr + util::ConfigValTInt::create(_configValMap, "activeworker", "maxLifetimeSecs", notReq, 60 * 60); + CVTIntPtr _monitorSleepTimeMilliSec = + util::ConfigValTInt::create(_configValMap, "activeworker", "monitorSleepTimeMilliSec", notReq, 15'000); + + // UberJobs + CVTIntPtr _uberJobMaxChunks = + util::ConfigValTInt::create(_configValMap, "uberjob", "maxChunks", notReq, 1000); + + /// This may impact `_resultMaxHttpConnections` as too many connections may cause kernel memory issues. + CVTIntPtr _commandMaxHttpConnections = + util::ConfigValTInt::create(_configValMap, "uberjob", "commandMaxHttpConnections", notReq, 2000); }; } // namespace lsst::qserv::cconfig diff --git a/src/ccontrol/CMakeLists.txt b/src/ccontrol/CMakeLists.txt index 625ccd331a..71a2bd2e38 100644 --- a/src/ccontrol/CMakeLists.txt +++ b/src/ccontrol/CMakeLists.txt @@ -31,7 +31,6 @@ target_link_libraries(ccontrol PUBLIC parser replica sphgeom - xrdreq XrdCl ) @@ -54,7 +53,6 @@ FUNCTION(ccontrol_tests) qmeta query rproc - xrdreq Boost::unit_test_framework Threads::Threads ) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 3e79478139..cc0da0f783 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -159,6 +159,7 @@ string readHttpFileAndMerge(string const& httpUrl, size_t fileSize, return string(); } + std::tuple readHttpFileAndMergeHttp( lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, function const& messageIsReady, @@ -330,6 +331,7 @@ MergingHandler::MergingHandler(std::shared_ptr merger, std: MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << _tableName); } + bool MergingHandler::flush(proto::ResponseSummary const& resp) { _wName = resp.wname(); @@ -381,18 +383,12 @@ void MergingHandler::errorFlush(std::string const& msg, int code) { LOGS(_log, LOG_LVL_ERROR, "Error receiving result."); } -bool MergingHandler::finished() const { return _flushed; } -bool MergingHandler::reset() { - // If we've pushed any bits to the merger successfully, we have to undo them - // to reset to a fresh state. For now, we will just fail if we've already - // begun merging. If we implement the ability to retract a partial result - // merge, then we can use it and do something better. - if (_flushed) { - return false; // Can't reset if we have already pushed state. - } - _initState(); - return true; +// Note that generally we always have an _infileMerger object except during +// a unit test. I suppose we could try to figure out how to create one. +// +void MergingHandler::prepScrubResults(int jobId, int attemptCount) { + if (_infileMerger) _infileMerger->prepScrub(jobId, attemptCount); } std::ostream& MergingHandler::print(std::ostream& os) const { @@ -504,12 +500,11 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe // This is needed to ensure the job query would be staying alive for the duration // of the operation to prevent inconsistency within the application. - auto const jobBase = getJobBase().lock(); - if (jobBase == nullptr) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobBase was NULL"); + auto const uberJob = getUberJob().lock(); + if (uberJob == nullptr) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, uberJob was NULL"); return {success, shouldCancel}; // both should still be false } - auto const uberJob = std::dynamic_pointer_cast(jobBase); LOGS(_log, LOG_LVL_TRACE, "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 8c0e8dc239..b7c7343e79 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -82,11 +82,6 @@ class MergingHandler : public qdisp::ResponseHandler { /// Signal an unrecoverable error condition. No further calls are expected. void errorFlush(std::string const& msg, int code) override; - /// @return true if the receiver has completed its duties. - bool finished() const override; - - bool reset() override; ///< Reset the state that a request can be retried. - /// Print a string representation of the receiver to an ostream std::ostream& print(std::ostream& os) const override; diff --git a/src/ccontrol/UserQueryAsyncResult.cc b/src/ccontrol/UserQueryAsyncResult.cc index 6cde104a2f..6ba3b0da06 100644 --- a/src/ccontrol/UserQueryAsyncResult.cc +++ b/src/ccontrol/UserQueryAsyncResult.cc @@ -85,6 +85,8 @@ void UserQueryAsyncResult::submit() { // if there are messages already it means the error was detected, stop right here if (_messageStore->messageCount() > 0) { + LOGS(_log, LOG_LVL_WARN, + "UserQueryAsyncResult::submit giving up, messageCount=" << _messageStore->messageCount()); return; } @@ -92,6 +94,7 @@ void UserQueryAsyncResult::submit() { if (_qInfo.czarId() != _czarId) { // TODO: tell user which czar was it? std::string message = "Query originated from different czar"; + LOGS(_log, LOG_LVL_WARN, "UserQueryAsyncResult::submit giving up, message=" << message); _messageStore->addErrorMessage("SYSTEM", message); return; } diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index a5fcf1fc90..8d101bf32a 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -68,6 +68,7 @@ #include "rproc/InfileMerger.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" +#include "util/QdispPool.h" namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.ccontrol.UserQueryFactory"); @@ -259,7 +260,7 @@ UserQueryFactory::~UserQueryFactory() { } UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::string const& defaultDb, - qdisp::SharedResources::Ptr const& qdispSharedResources, + util::QdispPool::Ptr const& qdispPool, std::string const& userQueryId, std::string const& msgTableName, std::string const& resultDb) { // result location could potentially be specified by SUBMIT command, for now @@ -365,6 +366,9 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st infileMergerConfig->debugNoMerge = _debugNoMerge; } + auto czarConfig = cconfig::CzarConfig::instance(); + int uberJobMaxChunks = czarConfig->getUberJobMaxChunks(); + // This, effectively invalid, UserQuerySelect object should report errors from both `errorExtra` // and errors that the QuerySession `qs` has stored internally. auto uq = std::make_shared( diff --git a/src/ccontrol/UserQueryFactory.h b/src/ccontrol/UserQueryFactory.h index a467ea07ad..90e5109794 100644 --- a/src/ccontrol/UserQueryFactory.h +++ b/src/ccontrol/UserQueryFactory.h @@ -42,7 +42,7 @@ // Local headers #include "global/stringTypes.h" -#include "qdisp/SharedResources.h" +#include "util/QdispPool.h" namespace lsst::qserv::ccontrol { class UserQuery; @@ -82,7 +82,7 @@ class UserQueryFactory : private boost::noncopyable { /// @param msgTableName: Name of the message table without database name. /// @return new UserQuery object std::shared_ptr newUserQuery(std::string const& query, std::string const& defaultDb, - qdisp::SharedResources::Ptr const& qdispSharedResources, + std::shared_ptr const& qdispPool, std::string const& userQueryId, std::string const& msgTableName, std::string const& resultDb); diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 8a51f8e597..c294f82674 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -71,7 +71,6 @@ // Third-party headers #include -#include "qdisp/QdispPool.h" // LSST headers #include "lsst/log/Log.h" @@ -86,7 +85,6 @@ #include "global/constants.h" #include "global/LogContext.h" #include "proto/worker.pb.h" -#include "proto/ProtoImporter.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" #include "qmeta/MessageStore.h" @@ -97,7 +95,6 @@ #include "qproc/geomAdapter.h" #include "qproc/IndexMap.h" #include "qproc/QuerySession.h" -#include "qproc/TaskMsgFactory.h" #include "query/ColumnRef.h" #include "query/FromList.h" #include "query/JoinRef.h" @@ -110,8 +107,9 @@ #include "sql/Schema.h" #include "util/Bug.h" #include "util/IterableFormatter.h" +#include "util/Histogram.h" //&&& +#include "util/QdispPool.h" #include "util/ThreadPriority.h" -#include "xrdreq/QueryManagementAction.h" #include "qdisp/UberJob.h" namespace { @@ -122,15 +120,6 @@ using namespace std; namespace lsst::qserv { -/// A class that can be used to parameterize a ProtoImporter for -/// debugging purposes -class ProtoPrinter { -public: - ProtoPrinter() {} - virtual void operator()(std::shared_ptr m) { std::cout << "Got taskmsg ok"; } - virtual ~ProtoPrinter() {} -}; - //////////////////////////////////////////////////////////////////////// // UserQuerySelect implementation namespace ccontrol { @@ -144,7 +133,8 @@ UserQuerySelect::UserQuerySelect(std::shared_ptr const& qs, std::shared_ptr const& secondaryIndex, std::shared_ptr const& queryMetadata, std::shared_ptr const& queryProgress, qmeta::CzarId czarId, - std::string const& errorExtra, bool async, std::string const& resultDb) + std::string const& errorExtra, bool async, std::string const& resultDb, + int uberJobMaxChunks) : _qSession(qs), _messageStore(messageStore), _executive(executive), @@ -156,7 +146,8 @@ UserQuerySelect::UserQuerySelect(std::shared_ptr const& qs, _czarId(czarId), _errorExtra(errorExtra), _resultDb(resultDb), - _async(async) {} + _async(async), + _uberJobMaxChunks(uberJobMaxChunks) {} std::string UserQuerySelect::getError() const { std::string div = (_errorExtra.size() && _qSession->getError().size()) ? " " : ""; @@ -251,7 +242,6 @@ void UserQuerySelect::submit() { LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect beginning submission"); assert(_infileMerger); - auto taskMsgFactory = std::make_shared(); _ttn = std::make_shared(_qMetaQueryId, _qSession->getOriginal()); std::vector chunks; std::mutex chunksMtx; @@ -271,6 +261,7 @@ void UserQuerySelect::submit() { } _executive->setScanInteractive(_qSession->getScanInteractive()); + _executive->setScanInfo(_qSession->getScanInfo()); string dbName(""); bool dbNameSet = false; @@ -284,29 +275,34 @@ void UserQuerySelect::submit() { &taskMsgFactory](util::CmdData*) { QSERV_LOGCONTEXT_QUERY(_queryId); - bool const fillInChunkIdTag = false; - qproc::ChunkQuerySpec::Ptr const cs = - _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec, fillInChunkIdTag); - std::string chunkResultName = ttn.make(cs->chunkId); - - ResourceUnit ru; - ru.setAsDbChunk(cs->db, cs->chunkId); - qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( - _czarId, _executive->getId(), sequence, ru, - std::make_shared(_infileMerger, chunkResultName), taskMsgFactory, cs, - chunkResultName); - _executive->add(jobDesc); - }; + // TODO:UJ The template(s) is generated here and later it is compared to other + // templates. It would be better to create the list of query templates here + // and just store the index into the list of templates in the `cs`. + qproc::ChunkQuerySpec::Ptr cs; + { + std::lock_guard lock(chunksMtx); + bool fillInChunkIdTag = false; // do not fill in the chunkId + cs = _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec, fillInChunkIdTag); + chunks.push_back(cs->chunkId); + } + std::string chunkResultName = _ttn->make(cs->chunkId); - if (!uberJobsEnabled) { - // references in captures cause races - auto funcBuildJob = [this, job{move(job)}](util::CmdData*) { - QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); - _executive->runJobQuery(job); - }; - auto cmd = std::make_shared(funcBuildJob); - _executive->queueJobStart(cmd); + // This should only need to be set once as all jobs should have the same database name. + if (cs->db != dbName) { + if (dbNameSet) { + LOGS(_log, LOG_LVL_ERROR, "dbName change from " << dbName << " to " << cs->db); + return; + } + dbName = cs->db; + dbNameSet = true; } + + ResourceUnit ru; + ru.setAsDbChunk(cs->db, cs->chunkId); + qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( + _qMetaCzarId, _executive->getId(), sequence, ru, + std::make_shared(_infileMerger, chunkResultName), cs, chunkResultName); + auto job = _executive->add(jobDesc); ++sequence; } @@ -315,25 +311,21 @@ void UserQuerySelect::submit() { } /// At this point the executive has a map of all jobs with the chunkIds as the key. - if (uberJobsEnabled) { - // TODO:UJ _maxCHunksPerUberJob maybe put in config??? or set on command line?? - // Different queries may benefit from different values - // Such as LIMIT=1 may work best with this at 1, where - // 100 would be better for others. - _maxChunksPerUberJob = 2; - // This is needed to prevent Czar::_monitor from starting things before they are ready. - _executive->setReadyToExecute(); - buildAndSendUberJobs(); - } + // This is needed to prevent Czar::_monitor from starting things before they are ready. + _executive->setReadyToExecute(); + buildAndSendUberJobs(); LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); // TODO:UJ Waiting for all jobs to start may not be needed anymore? _executive->waitForAllJobsToStart(); } +util::HistogramRolling histoBuildAndS("&&&uj histoBuildAndS", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + void UserQuerySelect::buildAndSendUberJobs() { + // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); - LOGS(_log, LOG_LVL_DEBUG, funcN << " start"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start " << _uberJobMaxChunks); // Ensure `_monitor()` doesn't do anything until everything is ready. if (!_executive->isReadyToExecute()) { @@ -343,16 +335,22 @@ void UserQuerySelect::buildAndSendUberJobs() { // Only one thread should be generating UberJobs for this user query at any given time. lock_guard fcLock(_buildUberJobMtx); - bool const clearFlag = false; - _executive->setFlagFailedUberJob(clearFlag); LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << _executive->getTotalJobs()); vector uberJobs; + qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); + if (unassignedChunksInQuery.empty()) { + LOGS(_log, LOG_LVL_DEBUG, funcN << " no unassigned Jobs"); + return; + } + + // Get czar info and the worker contactMap. auto czarPtr = czar::Czar::getCzar(); auto czFamilyMap = czarPtr->getCzarFamilyMap(); auto czChunkMap = czFamilyMap->getChunkMap(_queryDbName); auto czRegistry = czarPtr->getCzarRegistry(); + auto const wContactMap = czRegistry->waitForWorkerContactMap(); if (czChunkMap == nullptr) { LOGS(_log, LOG_LVL_ERROR, funcN << " no map found for queryDbName=" << _queryDbName); @@ -363,9 +361,7 @@ void UserQuerySelect::buildAndSendUberJobs() { auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); // Make a map of all jobs in the executive. - // TODO:UJ Maybe a check should be made that all datbases are in the same family? - - qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); + // TODO:UJ Maybe a check should be made that all databases are in the same family? // keep cycling through workers until no more chunks to place. // - create a map of UberJobs key=, val=> @@ -379,72 +375,111 @@ void UserQuerySelect::buildAndSendUberJobs() { // - For failures - If a worker cannot be contacted, that's an uberjob failure. // - uberjob failures (due to communications problems) will result in the uberjob // being broken up into multiple UberJobs going to different workers. - // - The best way to do this is probably to just kill the UberJob and mark all - // Jobs that were in that UberJob as needing re-assignment, and re-running - // the code here. The trick is going to be figuring out which workers are alive. - // Maybe force a fresh lookup from the replicator Registry when an UberJob fails. - map> workerJobMap; + // - If an UberJob fails, the UberJob is killed and all the Jobs it contained + // are flagged as needing re-assignment and this function will be called + // again to put those Jobs in new UberJobs. Correctly re-assigning the + // Jobs requires accurate information from the registry about which workers + // are alive or dead. + struct WInfoAndUJPtr { + using Ptr = shared_ptr; + qdisp::UberJob::Ptr uberJobPtr; + protojson::WorkerContactInfo::Ptr wInf; + }; + map workerJobMap; vector missingChunks; + auto startassign = CLOCK::now(); //&&& // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in - // numerical order. The workers run shared scans in numerical order of chunk id numbers. - // This keeps the number of partially complete UberJobs running on a worker to a minimum, + // numerical order. The workers run shared scans in numerical order of chunkId numbers. + // Numerical order keeps the number of partially complete UberJobs running on a worker to a minimum, // and should minimize the time for the first UberJob on the worker to complete. for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { + + bool const increaseAttemptCount = true; + jqPtr->getDescription()->incrAttemptCount(_executive, increaseAttemptCount); + + // If too many workers are down, there will be a chunk that cannot be found. + // Just continuing should leave jobs `unassigned` with their attempt count + // increased. Either the chunk will be found and jobs assigned, or the jobs' + // attempt count will reach max and the query will be cancelled + auto lambdaMissingChunk = [&](string const& msg) { + missingChunks.push_back(chunkId); + //&&&bool const increaseAttemptCount = true; + //&&&jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); + LOGS(_log, LOG_LVL_ERROR, msg); + }; + auto iter = chunkMapPtr->find(chunkId); if (iter == chunkMapPtr->end()) { - missingChunks.push_back(chunkId); - bool const increaseAttemptCount = true; - jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); - // Assign as many jobs as possible. Any chunks not found will be attempted later. + lambdaMissingChunk(funcN + " No chunkData for=" + to_string(chunkId)); continue; } czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - // TODO:UJ maybe if (targetWorker == nullptr || this worker already tried for this chunk) { - if (targetWorker == nullptr) { - LOGS(_log, LOG_LVL_ERROR, funcN << " No primary scan worker for chunk=" << chunkData->dump()); + // TODO:UJ maybe if (targetWorker == nullptr || ... || this worker already tried for this chunk) { + if (targetWorker == nullptr || targetWorker->isDead()) { + LOGS(_log, LOG_LVL_WARN, + funcN << " No primary scan worker for chunk=" + chunkData->dump() + << ((targetWorker == nullptr) ? " targ was null" : " targ was dead")); // Try to assign a different worker to this job auto workerHasThisChunkMap = chunkData->getWorkerHasThisMapCopy(); bool found = false; for (auto wIter = workerHasThisChunkMap.begin(); wIter != workerHasThisChunkMap.end() && !found; ++wIter) { auto maybeTarg = wIter->second.lock(); - if (maybeTarg != nullptr) { + if (maybeTarg != nullptr && !maybeTarg->isDead()) { targetWorker = maybeTarg; found = true; LOGS(_log, LOG_LVL_WARN, - funcN << " Alternate worker found for chunk=" << chunkData->dump()); + funcN << " Alternate worker=" << targetWorker->getWorkerId() + << " found for chunk=" << chunkData->dump()); } } if (!found) { - // If too many workers are down, there will be a chunk that cannot be found. - // Just continuing should leave jobs `unassigned` with their attempt count - // increased. Either the chunk will be found and jobs assigned, or the jobs' - // attempt count will reach max and the query will be cancelled - // TODO:UJ Needs testing/verification - LOGS(_log, LOG_LVL_ERROR, - funcN << " No primary or alternate worker found for chunk=" << chunkData->dump()); + lambdaMissingChunk(funcN + + " No primary or alternate worker found for chunk=" + chunkData->dump()); continue; } } // Add this job to the appropriate UberJob, making the UberJob if needed. string workerId = targetWorker->getWorkerId(); - auto& ujVect = workerJobMap[workerId]; - if (ujVect.empty() || ujVect.back()->getJobCount() >= _maxChunksPerUberJob) { + WInfoAndUJPtr::Ptr& wInfUJ = workerJobMap[workerId]; + if (wInfUJ == nullptr) { + wInfUJ = make_shared(); + auto iter = wContactMap->find(workerId); + if (iter == wContactMap->end()) { + // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. + // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, + // so maybe return internal error to the user? + throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + workerId); + } + wInfUJ->wInf = iter->second; + } + + if (wInfUJ->uberJobPtr == nullptr) { auto ujId = _uberJobIdSeq++; // keep ujId consistent string uberResultName = _ttn->make(ujId); auto respHandler = make_shared(_infileMerger, uberResultName); auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), ujId, _qMetaCzarId, targetWorker); - ujVect.push_back(uJob); + uJob->setWorkerContactInfo(wInfUJ->wInf); + wInfUJ->uberJobPtr = uJob; + }; + + wInfUJ->uberJobPtr->addJob(jqPtr); + + if (wInfUJ->uberJobPtr->getJobCount() >= _uberJobMaxChunks) { + // Queue the UberJob to be sent to a worker + _executive->addAndQueueUberJob(wInfUJ->uberJobPtr); + + // Clear the pinter so a new UberJob is created later if needed. + wInfUJ->uberJobPtr = nullptr; } - auto& ujVectBack = ujVect.back(); - ujVectBack->addJob(jqPtr); - LOGS(_log, LOG_LVL_DEBUG, - funcN << " ujVectBack{" << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() - << "}"); } + auto endassign = CLOCK::now(); //&&& + std::chrono::duration secsassign = endassign - startassign; // &&& + histoBuildAndS.addEntry(endassign, secsassign.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoBuildAndS.getString("")); if (!missingChunks.empty()) { string errStr = funcN + " a worker could not be found for these chunks "; @@ -453,31 +488,19 @@ void UserQuerySelect::buildAndSendUberJobs() { } errStr += " they will be retried later."; LOGS(_log, LOG_LVL_ERROR, errStr); - // There are likely to be unassigned jobs, so set a flag to try to make - // new uber jobs for these jobs. - _executive->setFlagFailedUberJob(true); } - // Add worker contact info to UberJobs. - auto const wContactMap = czRegistry->getWorkerContactMap(); - LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); - for (auto const& [wIdKey, ujVect] : workerJobMap) { - auto iter = wContactMap->find(wIdKey); - if (iter == wContactMap->end()) { - // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. - // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, - // so maybe return internal error to the user? - throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + wIdKey); - } - auto const& wContactInfo = iter->second; - for (auto const& ujPtr : ujVect) { - ujPtr->setWorkerContactInfo(wContactInfo); - } - _executive->addUberJobs(ujVect); - for (auto const& ujPtr : ujVect) { - _executive->runUberJob(ujPtr); + // Queue unqued UberJobs, these have less than the max number of jobs. + for (auto const& [wIdKey, winfUjPtr] : workerJobMap) { + if (winfUjPtr != nullptr) { + auto& ujPtr = winfUjPtr->uberJobPtr; + if (ujPtr != nullptr) { + _executive->addAndQueueUberJob(ujPtr); + } } } + + LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); } /// Block until a submit()'ed query completes. @@ -514,34 +537,27 @@ QueryState UserQuerySelect::join() { // finalRows < 0 indicates there was no postprocessing, so collected rows and final rows should be the // same. if (finalRows < 0) finalRows = collectedRows; - // Notify workers on the query completion/cancellation to ensure - // resources are properly cleaned over there as well. - proto::QueryManagement::Operation operation = proto::QueryManagement::COMPLETE; + QueryState state = SUCCESS; if (successful) { _qMetaUpdateStatus(qmeta::QInfo::COMPLETED, collectedRows, collectedBytes, finalRows); - LOGS(_log, LOG_LVL_INFO, "Joined everything (success)"); + LOGS(_log, LOG_LVL_INFO, "Joined everything (success) QID=" << getQueryId()); } else if (_killed) { // status is already set to ABORTED - LOGS(_log, LOG_LVL_ERROR, "Joined everything (killed)"); - operation = proto::QueryManagement::CANCEL; + LOGS(_log, LOG_LVL_ERROR, "Joined everything (killed) QID=" << getQueryId()); state = ERROR; } else { auto const status = resultSizeLimitExceeded ? qmeta::QInfo::FAILED_LR : qmeta::QInfo::FAILED; _qMetaUpdateStatus(status, collectedRows, collectedBytes, finalRows); - LOGS(_log, LOG_LVL_ERROR, "Joined everything (failure!)"); + LOGS(_log, LOG_LVL_ERROR, "Joined everything (failure!) QID=" << getQueryId()); operation = proto::QueryManagement::CANCEL; state = ERROR; } auto const czarConfig = cconfig::CzarConfig::instance(); - if (czarConfig->notifyWorkersOnQueryFinish()) { - try { - xrdreq::QueryManagementAction::notifyAllWorkers(czarConfig->getXrootdFrontendUrl(), operation, - _czarId, _queryId); - } catch (std::exception const& ex) { - LOGS(_log, LOG_LVL_WARN, ex.what()); - } - } + + // Notify workers on the query completion/cancellation to ensure + // resources are properly cleaned over there as well. + czar::Czar::getCzar()->getActiveWorkerMap()->addToDoneDeleteFiles(_executive->getId()); return state; } diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index bb4910b44b..6447f8fcd5 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -42,7 +42,6 @@ // Qserv headers #include "ccontrol/UserQuery.h" #include "css/StripingParams.h" -#include "qdisp/SharedResources.h" #include "qmeta/QInfo.h" #include "qmeta/types.h" #include "qproc/ChunkSpec.h" @@ -90,7 +89,8 @@ class UserQuerySelect : public UserQuery { std::shared_ptr const& secondaryIndex, std::shared_ptr const& queryMetadata, std::shared_ptr const& queryProgress, qmeta::CzarId czarId, - std::string const& errorExtra, bool async, std::string const& resultDb); + std::string const& errorExtra, bool async, std::string const& resultDb, + int uberJobMaxChunks); UserQuerySelect(UserQuerySelect const&) = delete; UserQuerySelect& operator=(UserQuerySelect const&) = delete; @@ -194,10 +194,8 @@ class UserQuerySelect : public UserQuery { std::string _resultDb; ///< Result database TODO:UJ same as resultLoc??) bool _async; ///< true for async query - /// TODO:UJ The maximum number of chunks allowed in an UberJob. At the very - /// least, this needs to be set in the configuration. However, it may also - /// be useful to change this based on the nature of each UserQuery. - int _maxChunksPerUberJob = 1; + /// The maximum number of chunks allowed in an UberJob, set from config. + int const _uberJobMaxChunks; std::atomic _uberJobIdSeq{1}; ///< Sequence number for UberJobs in this query. std::shared_ptr _ttn; ///< Temporary table name generator. diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc new file mode 100644 index 0000000000..68f7be0923 --- /dev/null +++ b/src/czar/ActiveWorker.cc @@ -0,0 +1,311 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "czar/ActiveWorker.h" + +// System headers +#include + +// Qserv headers +#include "cconfig/CzarConfig.h" +#include "czar/Czar.h" +#include "http/Client.h" +#include "http/MetaModule.h" +#include "util/common.h" +#include "util/QdispPool.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.ActiveWorker"); +} // namespace + +namespace lsst::qserv::czar { + +string ActiveWorker::getStateStr(State st) { + switch (st) { + case ALIVE: + return string("ALIVE"); + case QUESTIONABLE: + return string("QUESTIONABLE"); + case DEAD: + return string("DEAD"); + } + return string("unknown"); +} + +bool ActiveWorker::compareContactInfo(protojson::WorkerContactInfo const& wcInfo) const { + lock_guard lg(_aMtx); + auto wInfo_ = _wqsData->getWInfo(); + if (wInfo_ == nullptr) return false; + return wInfo_->isSameContactInfo(wcInfo); +} + +void ActiveWorker::setWorkerContactInfo(protojson::WorkerContactInfo::Ptr const& wcInfo) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " new info=" << wcInfo->dump()); + lock_guard lg(_aMtx); + _wqsData->setWInfo(wcInfo); +} + +void ActiveWorker::_changeStateTo(State newState, double secsSinceUpdate, string const& note) { + auto lLvl = (newState == DEAD) ? LOG_LVL_ERROR : LOG_LVL_INFO; + LOGS(_log, lLvl, + note << " oldState=" << getStateStr(_state) << " newState=" << getStateStr(newState) + << " secsSince=" << secsSinceUpdate); + _state = newState; +} + +void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, + double maxLifetime) { + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " start"); + bool newlyDeadWorker = false; + protojson::WorkerContactInfo::Ptr wInfo_; + { + lock_guard lg(_aMtx); + wInfo_ = _wqsData->getWInfo(); + if (wInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no WorkerContactInfo"); + return; + } + double secsSinceUpdate = wInfo_->timeSinceRegUpdateSeconds(); + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << " wInfo=" << wInfo_->dump() + << " secsSince=" << wInfo_->timeSinceRegUpdateSeconds() + << " secsSinceUpdate=" << secsSinceUpdate); + + // Update the last time the registry contacted this worker. + // TODO:UJ - This needs to be added to the dashboard. + switch (_state) { + case ALIVE: { + if (secsSinceUpdate >= timeoutAliveSecs) { + _changeStateTo(QUESTIONABLE, secsSinceUpdate, cName(__func__)); + } + break; + } + case QUESTIONABLE: { + if (secsSinceUpdate < timeoutAliveSecs) { + _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); + } + if (secsSinceUpdate >= timeoutDeadSecs) { + _changeStateTo(DEAD, secsSinceUpdate, cName(__func__)); + // All uberjobs for this worker need to die. + newlyDeadWorker = true; + } + break; + } + case DEAD: { + if (secsSinceUpdate < timeoutAliveSecs) { + _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); + } else { + // Don't waste time on this worker until the registry has heard from it. + return; + } + break; + } + } + } + + // _aMtx must not be held when calling this. + if (newlyDeadWorker) { + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " worker " << wInfo_->wId << " appears to have died, reassigning its jobs."); + czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInfo_->wId); + } + + shared_ptr jsWorkerReqPtr; + { + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // message to send to the worker. + jsWorkerReqPtr = _wqsData->serializeJson(maxLifetime); + } + + // Always send the message as it's a way to inform the worker that this + // czar is functioning and capable of receiving requests. + Ptr thisPtr = shared_from_this(); + auto sendStatusMsgFunc = [thisPtr, wInfo_, jsWorkerReqPtr](util::CmdData*) { + thisPtr->_sendStatusMsg(wInfo_, jsWorkerReqPtr); + }; + + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(sendStatusMsgFunc)); + auto qdisppool = czar::Czar::getCzar()->getQdispPool(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " queuing message"); + qdisppool->queCmd(cmd, 1); +} + +void ActiveWorker::_sendStatusMsg(protojson::WorkerContactInfo::Ptr const& wInf, + std::shared_ptr const& jsWorkerReqPtr) { + auto& jsWorkerReq = *jsWorkerReqPtr; + auto const method = http::Method::POST; + if (wInf == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo was null."); + return; + } + auto [ciwId, ciwHost, ciwManag, ciwPort] = wInf->getAll(); + string const url = "http://" + ciwHost + ":" + to_string(ciwPort) + "/querystatus"; + vector const headers = {"Content-Type: application/json"}; + auto const& czarConfig = cconfig::CzarConfig::instance(); + + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << jsWorkerReq); + string const requestContext = "Czar: '" + http::method2string(method) + "' stat request to '" + url + "'"; + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << " czarPost url=" << url << " request=" << jsWorkerReq.dump() + << " headers=" << headers[0]); + http::Client client(method, url, jsWorkerReq.dump(), headers); + bool transmitSuccess = false; + string exceptionWhat; + json response; + try { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read start"); + response = client.readAsJson(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read end"); + if (0 != response.at("success").get()) { + bool startupTimeChanged = false; + startupTimeChanged = _wqsData->handleResponseJson(response); + transmitSuccess = true; + if (startupTimeChanged) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " worker startupTime changed, likely rebooted."); + // kill all incomplete UberJobs on this worker. + czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInf->wId); + } + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure response success=0 " << response); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_ERROR, requestContext + " transmit failure, ex: " + ex.what()); + exceptionWhat = ex.what(); + } + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " transmit failure " << jsWorkerReq.dump() << " resp=" << response); + } +} + +void ActiveWorker::addToDoneDeleteFiles(QueryId qId) { _wqsData->addToDoneDeleteFiles(qId); } + +void ActiveWorker::addToDoneKeepFiles(QueryId qId) { _wqsData->addToDoneKeepFiles(qId); } + +void ActiveWorker::removeDeadUberJobsFor(QueryId qId) { _wqsData->removeDeadUberJobsFor(qId); } + +void ActiveWorker::addDeadUberJob(QueryId qId, UberJobId ujId) { + auto now = CLOCK::now(); + _wqsData->addDeadUberJob(qId, ujId, now); +} + +protojson::WorkerContactInfo::Ptr ActiveWorker::getWInfo() const { + std::lock_guard lg(_aMtx); + if (_wqsData == nullptr) return nullptr; + return _wqsData->getWInfo(); +} + +ActiveWorker::State ActiveWorker::getState() const { + std::lock_guard lg(_aMtx); + return _state; +} + +string ActiveWorker::dump() const { + lock_guard lg(_aMtx); + return _dump(); +} + +string ActiveWorker::_dump() const { + stringstream os; + os << "ActiveWorker " << (_wqsData->dump()); + return os.str(); +} + +ActiveWorkerMap::ActiveWorkerMap(std::shared_ptr const& czarConfig) + : _timeoutAliveSecs(czarConfig->getActiveWorkerTimeoutAliveSecs()), + _timeoutDeadSecs(czarConfig->getActiveWorkerTimeoutDeadSecs()), + _maxLifetime(czarConfig->getActiveWorkerMaxLifetimeSecs()) {} + +void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap, + protojson::CzarContactInfo::Ptr const& czInfo, + std::string const& replicationInstanceId, + std::string const& replicationAuthKey) { + // Go through wcMap, update existing entries in _awMap, create new entries for those that don't exist, + lock_guard awLg(_awMapMtx); + for (auto const& [wcKey, wcVal] : wcMap) { + auto iter = _awMap.find(wcKey); + if (iter == _awMap.end()) { + auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " AciveWorker created for " << wcKey); + _awMap[wcKey] = newAW; + if (_czarCancelAfterRestart) { + newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); + } + } else { + auto aWorker = iter->second; + if (!aWorker->compareContactInfo(*wcVal)) { + // This should not happen, but try to handle it gracefully if it does. + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " worker contact info changed for " << wcKey + << " new=" << wcVal->dump() << " old=" << aWorker->dump()); + // If there is existing information, only host and port values will change. + aWorker->setWorkerContactInfo(wcVal); + } + aWorker->getWInfo()->setRegUpdateTime(wcVal->getRegUpdateTime()); + } + } +} + +void ActiveWorkerMap::setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { + _czarCancelAfterRestart = true; + _czarCancelAfterRestartCzId = czId; + _czarCancelAfterRestartQId = lastQId; +} + +ActiveWorker::Ptr ActiveWorkerMap::getActiveWorker(string const& workerId) const { + lock_guard lck(_awMapMtx); + auto iter = _awMap.find(workerId); + if (iter == _awMap.end()) return nullptr; + return iter->second; +} + +void ActiveWorkerMap::sendActiveWorkersMessages() { + // Send messages to each active worker as needed + lock_guard lck(_awMapMtx); + for (auto&& [wName, awPtr] : _awMap) { + awPtr->updateStateAndSendMessages(_timeoutAliveSecs, _timeoutDeadSecs, _maxLifetime); + } +} + +void ActiveWorkerMap::addToDoneDeleteFiles(QueryId qId) { + lock_guard lck(_awMapMtx); + for (auto const& [wName, awPtr] : _awMap) { + awPtr->addToDoneDeleteFiles(qId); + awPtr->removeDeadUberJobsFor(qId); + } +} + +void ActiveWorkerMap::addToDoneKeepFiles(QueryId qId) { + lock_guard lck(_awMapMtx); + for (auto const& [wName, awPtr] : _awMap) { + awPtr->addToDoneKeepFiles(qId); + awPtr->removeDeadUberJobsFor(qId); + } +} + +} // namespace lsst::qserv::czar diff --git a/src/czar/ActiveWorker.h b/src/czar/ActiveWorker.h new file mode 100644 index 0000000000..d462f0d0e1 --- /dev/null +++ b/src/czar/ActiveWorker.h @@ -0,0 +1,251 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_CZAR_ACTIVEWORKER_H +#define LSST_QSERV_CZAR_ACTIVEWORKER_H + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "protojson/WorkerQueryStatusData.h" +#include "util/Bug.h" + +namespace lsst::qserv::cconfig { +class CzarConfig; +} + +// This header declarations +namespace lsst::qserv::czar { + +/// This class is used to track information important to the czar and a +/// specific worker. Primarily the czar cares about the worker being alive +/// and informing the worker that various query IDs and UberJobs +/// have finished or need to be cancelled. +/// - maintain list of done/cancelled queries for an active worker, and send +/// that list to the worker. Once the worker has accepted the list, remove +/// all of those queryId's from the lists. +/// - maintain a list of killed UberJobs. If an UberJob is killed, nothing +/// will every look for its files, so they should be deleted, and the +/// worker should avoid working on Tasks for that UberJob. +/// The only UberJob deaths that need to be sent to a worker is when +/// the czar kills an UberJob because the worker died/vanished, and +/// the only time this would be sent is when a worker came back from +/// the dead. +/// The reason this only applies to died/vanished workers is that all +/// other workers know their UberJobs are dead because the worker killed +/// them. If the worker isn't told, it will continue working on +/// the UberJob until it finishes, and then find out the UberJob was killed +/// when it tries to return results to the czar. The worker should delete +/// files for said UberJob at that point. +/// So, this should be very rare, only results in extra load. +/// +/// If a worker goes missing from the registry, it is considered DEAD and may be +/// removed after a period of time. +/// If a worker hasn't been heard from in (timeout period), it is considered QUESIONABLE. +/// If a QUESTIONABLE worker hasn't been heard from in (timeout period), its state is changed +/// to DEAD. +/// +/// When a worker becomes DEAD: (see Czar::_monitor). +/// - Affected UberJobs are killed. +/// - New UberJobs are built to handle unassigned jobs where dead workers are skipped and +/// the jobs are assigned to alternate workers. +/// +class ActiveWorker : public std::enable_shared_from_this { +public: + using Ptr = std::shared_ptr; + + enum State { ALIVE = 0, QUESTIONABLE, DEAD }; + + ActiveWorker() = delete; + ActiveWorker(ActiveWorker const&) = delete; + ActiveWorker& operator=(ActiveWorker const&) = delete; + + std::string cName(const char* fName) { + auto wqsd = _wqsData; + return std::string("ActiveWorker::") + fName + " " + ((wqsd == nullptr) ? "?" : wqsd->dump()); + } + + static std::string getStateStr(State st); + + static Ptr create(protojson::WorkerContactInfo::Ptr const& wInfo, + protojson::CzarContactInfo::Ptr const& czInfo, std::string const& replicationInstanceId, + std::string const& replicationAuthKey) { + return Ptr(new ActiveWorker(wInfo, czInfo, replicationInstanceId, replicationAuthKey)); + } + + /// This function should only be called before the _monitor thread is started + /// and shortly after czar startup: it tells all workers to delete all + /// query information for queries with czarId `czId` and queryId less than + /// or equal to `lastQId`. + void setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { + _wqsData->setCzarCancelAfterRestart(czId, lastQId); + } + + protojson::WorkerContactInfo::Ptr getWInfo() const; + + ~ActiveWorker() = default; + + /// Return true if there were differences in worker id, host, or port values. + bool compareContactInfo(protojson::WorkerContactInfo const& wcInfo) const; + + void setWorkerContactInfo(protojson::WorkerContactInfo::Ptr const& wcInfo); + + /// Check this workers state (by looking at contact information) and queue + /// the WorkerQueryStatusData message `_wqsData` to be sent if this worker + /// isn't DEAD. + void updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime); + + /// Add `qId` to list of QueryId's that the worker can discard all tasks and + /// result files for. This `qId` will be removed from the list once the worker + /// has responded to the `_wqsData` message with this `qId` in the appropriate + /// list. + /// It is expected that all completed or cancelled queries on this worker will + /// be added to this list. + void addToDoneDeleteFiles(QueryId qId); + + /// Add `qId` to list of QueryId's that the worker where the worker must hold + /// onto result files but tasks can be eliminated. This `qId` will be removed + /// from the list once the worker has responded to the `_wqsData` message with + /// this `qId` in the appropriate list. + void addToDoneKeepFiles(QueryId qId); + + /// Add the uberjob to the list of dead uberjobs. This `qId` will be removed + /// from the list once the worker has responded to the `_wqsData` message with + /// this `qId` in the appropriate list. Or the `qId` is in a + /// removeDeadUberJobsFor() call. + void addDeadUberJob(QueryId qId, UberJobId ujId); + + /// If a query is completed or cancelled, there's no reason to track the + /// individual UberJobs anymore, so this function will get rid of them. + void removeDeadUberJobsFor(QueryId qId); + + State getState() const; + + std::string dump() const; + +private: + ActiveWorker(protojson::WorkerContactInfo::Ptr const& wInfo, + protojson::CzarContactInfo::Ptr const& czInfo, std::string const& replicationInstanceId, + std::string const& replicationAuthKey) + : _wqsData(protojson::WorkerQueryStatusData::create(wInfo, czInfo, replicationInstanceId, + replicationAuthKey)) { + if (_wqsData == nullptr) { + throw util::Bug(ERR_LOC, "ActiveWorker _wqsData null"); + } + } + + /// Change the state to `newState` and log if it is different. + /// _aMtx must be held before calling. + void _changeStateTo(State newState, double secsSinceUpdate, std::string const& note); + + /// Send the `jsWorkerReqPtr` json message to the worker referenced by `wInf` to + /// transmit the `_wqsData` state. + void _sendStatusMsg(protojson::WorkerContactInfo::Ptr const& wInf, + std::shared_ptr const& jsWorkerReqPtr); + + /// Dump a log string for this object. + /// _aMtx must be held before calling. + std::string _dump() const; + + /// Contains data that needs to be sent to workers about finished/cancelled + /// user queries and UberJobs. It must not be null. + protojson::WorkerQueryStatusData::Ptr const _wqsData; + + State _state{QUESTIONABLE}; ///< current state of this worker. + + mutable std::mutex _aMtx; ///< protects _wInfo, _state, _qIdDoneKeepFiles, _qIdDoneDeleteFiles +}; + +/// This class maintains a list of all workers, indicating which are considered active. +/// Communication problems with workers could cause interesting race conditions, so +/// workers will remain on the list for a very long time after they have disappeared +/// in case they come back from the dead. +class ActiveWorkerMap { +public: + using Ptr = std::shared_ptr; + ActiveWorkerMap() = default; + ActiveWorkerMap(ActiveWorkerMap const&) = delete; + ActiveWorkerMap operator=(ActiveWorkerMap const&) = delete; + + ActiveWorkerMap(std::shared_ptr const& czarConfig); + + ~ActiveWorkerMap() = default; + + std::string cName(const char* fName) { return std::string("ActiveWorkerMap::") + fName + " "; } + + /// Use information gathered from the registry to update the map. The registry + /// contains last contact time (used for determining aliveness) and worker contact information. + void updateMap(protojson::WorkerContactInfo::WCMap const& wcMap, + protojson::CzarContactInfo::Ptr const& czInfo, std::string const& replicationInstanceId, + std::string const& replicationAuthKey); + + /// If this is to be called, it must be called before Czar::_monitor is started: + /// It tells the workers all queries from `czId` with QueryIds less than `lastQId` + /// should be cancelled. + void setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId); + + /// Return a pointer to the `ActiveWorker` associated with `workerId`. + ActiveWorker::Ptr getActiveWorker(std::string const& workerId) const; + + /// Call `updateStateAndSendMessages` for all workers in this map. + void sendActiveWorkersMessages(); + + /// Add `qId` to the list of query ids where the worker can throw away all related + /// Tasks and result files. This is used for all completed user queries and cancelled + /// user queries. + void addToDoneDeleteFiles(QueryId qId); + + /// Add `qId` to the list of query ids where the worker must hold onto result + /// files but all incomplete Tasks can be stopped. This is used for `rowLimitComplete` + /// where enough rows have been found to complete a user query with a LIMIT + /// clause. The czar may still need to collect the result files from the worker. + /// Once the czar has completed the user query, the `qId` will be added to + /// `addToDoneDeleteFiles` so the workers will delete the files. + void addToDoneKeepFiles(QueryId qId); + +private: + std::map _awMap; ///< Key is worker id. + mutable std::mutex _awMapMtx; ///< protects _awMap; + + /// @see CzarConfig::getActiveWorkerTimeoutAliveSecs() + double _timeoutAliveSecs = 60.0 * 5.0; + + /// @see CzarConfig::getActiveWorkerTimeoutDeadSecs() + double _timeoutDeadSecs = 60.0 * 10.0; + + /// @see CzarConfig::getActiveWorkerMaxLifetimeSecs() + double _maxLifetime = 60.0 * 60.0; + + bool _czarCancelAfterRestart = false; + CzarIdType _czarCancelAfterRestartCzId = 0; + QueryId _czarCancelAfterRestartQId = 0; +}; + +} // namespace lsst::qserv::czar + +#endif // LSST_QSERV_CZAR_ACTIVEWORKER_H diff --git a/src/czar/CMakeLists.txt b/src/czar/CMakeLists.txt index 15210908f3..e4025b79e9 100644 --- a/src/czar/CMakeLists.txt +++ b/src/czar/CMakeLists.txt @@ -1,6 +1,7 @@ add_library(czar SHARED) target_sources(czar PRIVATE + ActiveWorker.cc ChttpModule.cc Czar.cc CzarChunkMap.cc @@ -12,7 +13,6 @@ target_sources(czar PRIVATE HttpCzarQueryModule.cc HttpCzarSvc.cc HttpCzarWorkerModule.cc - HttpModule.cc HttpMonitorModule.cc HttpSvc.cc MessageTable.cc @@ -27,6 +27,7 @@ target_include_directories(czar PRIVATE target_link_libraries(czar PUBLIC cconfig http + protojson qdisp qhttp util diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 814c6ee826..0124f81856 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -41,6 +41,7 @@ #include "ccontrol/UserQueryResources.h" #include "ccontrol/UserQuerySelect.h" #include "ccontrol/UserQueryType.h" +#include "czar/ActiveWorker.h" #include "czar/CzarChunkMap.h" #include "czar/CzarErrors.h" #include "czar/CzarThreads.h" @@ -51,8 +52,6 @@ #include "proto/worker.pb.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" -#include "qdisp/QdispPool.h" -#include "qdisp/SharedResources.h" #include "qproc/DatabaseModels.h" #include "rproc/InfileMerger.h" #include "sql/SqlConnection.h" @@ -61,14 +60,11 @@ #include "util/common.h" #include "util/FileMonitor.h" #include "util/IterableFormatter.h" +#include "util/QdispPool.h" #include "util/String.h" -#include "xrdreq/QueryManagementAction.h" -#include "XrdSsi/XrdSsiProvider.hh" using namespace std; -extern XrdSsiProvider* XrdSsiProviderClient; - // This macro is used to convert empty strings into "0" in order to avoid // problems with calling std::atoi() when the string is empty. #define ZERO_IF_EMPTY_STR(x) ((x.empty()) ? "0" : (x)) @@ -82,6 +78,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.Czar"); namespace lsst::qserv::czar { Czar::Ptr Czar::_czar; +uint64_t const Czar::czarStartupTime = millisecSinceEpoch(CLOCK::now()); Czar::Ptr Czar::createCzar(string const& configFilePath, string const& czarName) { _czar.reset(new Czar(configFilePath, czarName)); @@ -90,17 +87,31 @@ Czar::Ptr Czar::createCzar(string const& configFilePath, string const& czarName) void Czar::_monitor() { string const funcN("Czar::_monitor"); + uint16_t loopCount = 0; // unsigned to wrap around while (_monitorLoop) { + ++loopCount; this_thread::sleep_for(_monitorSleepTime); LOGS(_log, LOG_LVL_DEBUG, funcN << " start0"); /// Check database for changes in worker chunk assignments and aliveness - _czarFamilyMap->read(); + try { + // TODO:UJ The read() is incredibly expensive until the database has + // a "changed" field of some kind (preferably timestamp) to + // indicate the last time it changed. + // For Now, just do one read every few times through this loop. + if (loopCount % 10 == 0 || true) { + _czarFamilyMap->read(); + } + } catch (ChunkMapException const& cmex) { + // There are probably chunks that don't exist on any alive worker, + // continue on in hopes that workers will show up with the missing chunks + // later. + LOGS(_log, LOG_LVL_ERROR, funcN << " family map read problems " << cmex.what()); + } - // TODO:UJ If there were changes in `_czarFamilyMap`, see if any - // workers went down. If any did, `_unassign` all Jobs in UberJobs - // for the downed workers. The `_unassigned` Jobs should get - // reassigned in the next section `assignJobsToUberJobs`. + // Send appropriate messages to all ActiveWorkers. This will + // check if workers have died by timeout. + _czarRegistry->sendActiveWorkersMessages(); /// Create new UberJobs (if possible) for all jobs that are /// unassigned for any reason. @@ -108,6 +119,7 @@ void Czar::_monitor() { { // Make a copy of all valid Executives lock_guard execMapLock(_executiveMapMtx); + // Use an iterator so it's easy/quick to delete dead weak pointers. auto iter = _executiveMap.begin(); while (iter != _executiveMap.end()) { auto qIdKey = iter->first; @@ -125,16 +137,15 @@ void Czar::_monitor() { execVal->assignJobsToUberJobs(); } - // TODO:UJ Maybe get missing results from workers. - // This would be files that workers sent messages to the czar to - // collect, but there was a communication problem and the czar didn't get the message - // or didn't collect the file. to retrieve complete files that haven't been - // collected. - // Basically, is there a reasonable way to check that all UberJobs are being handled - // and nothing has fallen through the cracks? + // To prevent anything from slipping through the cracks: + // Workers will keep trying to transmit results until they think the czar is dead. + // If a worker thinks the czar died, it will cancel all related jobs that it has, + // and if the czar sends a status message to that worker, that worker will send back + // a separate message (see WorkerCzarComIssue) saying it killed everything that this + // czar gave it. Upon getting this message from a worker, this czar will reassign + // everything it had sent to that worker. - // TODO:UJ Maybe send a list of cancelled and completed queries to the workers? - // How long should queryId's remain on this list? + // TODO:UJ How long should queryId's remain on this list? } } @@ -144,7 +155,9 @@ Czar::Czar(string const& configFilePath, string const& czarName) _czarConfig(cconfig::CzarConfig::create(configFilePath, czarName)), _idCounter(), _uqFactory(), - _clientToQuery() { + _clientToQuery(), + _monitorSleepTime (_czarConfig->getMonitorSleepTimeMilliSec()), + _activeWorkerMap(new ActiveWorkerMap(_czarConfig)) { // set id counter to milliseconds since the epoch, mod 1 year. struct timeval tv; gettimeofday(&tv, nullptr); @@ -161,28 +174,24 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->czarId); - try { - _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); - } catch (ChunkMapException const& exc) { - LOGS(_log, LOG_LVL_WARN, string(__func__) + " failed to create CzarChunkMap " + exc.what()); - } - // Tell workers to cancel any queries that were submitted before this restart of Czar. - // Figure out which query (if any) was recorded in Czar database before the restart. + // Figure out which query (if any) was recorded in Czar databases before the restart. // The id will be used as the high-watermark for queries that need to be cancelled. // All queries that have identifiers that are strictly less than this one will // be affected by the operation. // if (_czarConfig->notifyWorkersOnCzarRestart()) { try { - xrdreq::QueryManagementAction::notifyAllWorkers(_czarConfig->getXrootdFrontendUrl(), - proto::QueryManagement::CANCEL_AFTER_RESTART, - _czarConfig->id(), _lastQueryIdBeforeRestart()); + QueryId lastQId = _lastQueryIdBeforeRestart(); + _activeWorkerMap->setCzarCancelAfterRestart(_czarConfig->id(), lastQId); } catch (std::exception const& ex) { LOGS(_log, LOG_LVL_WARN, ex.what()); } } + // This will block until there is a successful read of the database tables. + _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + int qPoolSize = _czarConfig->getQdispPoolSize(); int maxPriority = std::max(0, _czarConfig->getQdispMaxPriority()); string vectRunSizesStr = _czarConfig->getQdispVectRunSizes(); @@ -190,26 +199,24 @@ Czar::Czar(string const& configFilePath, string const& czarName) string vectMinRunningSizesStr = _czarConfig->getQdispVectMinRunningSizes(); vector vectMinRunningSizes = util::String::parseToVectInt(vectMinRunningSizesStr, ":", 0); LOGS(_log, LOG_LVL_INFO, - "INFO qdisp config qPoolSize=" << qPoolSize << " maxPriority=" << maxPriority << " vectRunSizes=" - << vectRunSizesStr << " -> " << util::prettyCharList(vectRunSizes) - << " vectMinRunningSizes=" << vectMinRunningSizesStr << " -> " - << util::prettyCharList(vectMinRunningSizes)); - qdisp::QdispPool::Ptr qdispPool = - make_shared(qPoolSize, maxPriority, vectRunSizes, vectMinRunningSizes); - qdisp::CzarStats::setup(qdispPool); - - _qdispSharedResources = qdisp::SharedResources::create(qdispPool); + " qdisp config qPoolSize=" << qPoolSize << " maxPriority=" << maxPriority << " vectRunSizes=" + << vectRunSizesStr << " -> " << util::prettyCharList(vectRunSizes) + << " vectMinRunningSizes=" << vectMinRunningSizesStr << " -> " + << util::prettyCharList(vectMinRunningSizes)); + _qdispPool = make_shared(qPoolSize, maxPriority, vectRunSizes, vectMinRunningSizes); + qdisp::CzarStats::setup(_qdispPool); int xrootdCBThreadsMax = _czarConfig->getXrootdCBThreadsMax(); int xrootdCBThreadsInit = _czarConfig->getXrootdCBThreadsInit(); LOGS(_log, LOG_LVL_INFO, "config xrootdCBThreadsMax=" << xrootdCBThreadsMax); LOGS(_log, LOG_LVL_INFO, "config xrootdCBThreadsInit=" << xrootdCBThreadsInit); - XrdSsiProviderClient->SetCBThreads(xrootdCBThreadsMax, xrootdCBThreadsInit); int const xrootdSpread = _czarConfig->getXrootdSpread(); LOGS(_log, LOG_LVL_INFO, "config xrootdSpread=" << xrootdSpread); - XrdSsiProviderClient->SetSpread(xrootdSpread); _queryDistributionTestVer = _czarConfig->getQueryDistributionTestVer(); + _commandHttpPool = shared_ptr( + new http::ClientConnPool(_czarConfig->getCommandMaxHttpConnections())); + LOGS(_log, LOG_LVL_INFO, "Creating czar instance with name " << czarName); LOGS(_log, LOG_LVL_INFO, "Czar config: " << *_czarConfig); @@ -239,7 +246,7 @@ Czar::Czar(string const& configFilePath, string const& czarName) startGarbageCollectAsync(_czarConfig); startGarbageCollectInProgress(_czarConfig, _uqFactory->userQuerySharedResources()->czarId, _uqFactory->userQuerySharedResources()->queryMetadata); - _czarRegistry = CzarRegistry::create(_czarConfig); + _czarRegistry = CzarRegistry::create(_czarConfig, _activeWorkerMap); // Start the monitor thread thread monitorThrd(&Czar::_monitor, this); @@ -295,8 +302,7 @@ SubmitResult Czar::submitQuery(string const& query, map const& h ccontrol::UserQuery::Ptr uq; { lock_guard lock(_mutex); - uq = _uqFactory->newUserQuery(query, defaultDb, getQdispSharedResources(), userQueryId, msgTableName, - resultDb); + uq = _uqFactory->newUserQuery(query, defaultDb, getQdispPool(), userQueryId, msgTableName, resultDb); } // Add logging context with query ID @@ -657,4 +663,32 @@ std::shared_ptr Czar::getExecutiveFromMap(QueryId qId) { return exec; } +std::map> Czar::getExecMapCopy() const { + // Copy list of executives so the mutex isn't held forever. + std::map> execMap; + { + lock_guard lgMap(_executiveMapMtx); + execMap = _executiveMap; + } + return execMap; +} + +void Czar::killIncompleteUbjerJobsOn(std::string const& restartedWorkerId) { + // Copy list of executives so the mutex isn't held forever. + std::map> execMap; + { + lock_guard lgMap(_executiveMapMtx); + execMap = _executiveMap; + } + + // For each executive, go through its list of uberjobs and cancel those jobs + // with workerId == restartedWorkerId && + for (auto const& [eKey, wPtrExec] : execMap) { + auto exec = wPtrExec.lock(); + if (exec != nullptr) { + exec->killIncompleteUberJobsOnWorker(restartedWorkerId); + } + } +} + } // namespace lsst::qserv::czar diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 2e7ec41822..276be00beb 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -37,10 +37,10 @@ #include "ccontrol/UserQuery.h" #include "ccontrol/UserQueryFactory.h" #include "czar/SubmitResult.h" +#include "global/clock_defs.h" #include "global/intTypes.h" #include "global/stringTypes.h" #include "mysql/MySqlConfig.h" -#include "qdisp/SharedResources.h" #include "util/ConfigStore.h" #include "util/Timer.h" @@ -51,9 +51,14 @@ class CzarConfig; } // namespace lsst::qserv::cconfig namespace lsst::qserv::czar { +class ActiveWorkerMap; class HttpSvc; } // namespace lsst::qserv::czar +namespace lsst::qserv::http { +class ClientConnPool; +} // namespace lsst::qserv::http + namespace lsst::qserv::util { class FileMonitor; } // namespace lsst::qserv::util @@ -118,8 +123,9 @@ class Czar { */ static Ptr getCzar() { return _czar; } - /// Return a pointer to QdispSharedResources - qdisp::SharedResources::Ptr getQdispSharedResources() { return _qdispSharedResources; } + + /// Remove all old tables in the qservResult database. + void removeOldResultTables(); /// @return true if trivial queries should be treated as /// interactive queries to stress test the czar. @@ -139,6 +145,26 @@ class Czar { /// Get the executive associated with `qId`, this may be nullptr. std::shared_ptr getExecutiveFromMap(QueryId qId); + std::shared_ptr getActiveWorkerMap() const { return _activeWorkerMap; } + + std::map> getExecMapCopy() const; + + /// This function kills incomplete UberJobs associated with `workerId`. + /// This is done when it is believed a worker has died. The executive + /// un-assignes the Jobs associated with the UberJobs and then + /// adds the ids to lists for the affected worker. If the worker + /// reconnects, it will stop work on those UberJobs when it gets the + /// list. + void killIncompleteUbjerJobsOn(std::string const& workerId); + + std::shared_ptr getQdispPool() const { return _qdispPool; } + + std::shared_ptr getCommandHttpPool() const { return _commandHttpPool; } + + /// Startup time of czar, sent to workers so they can detect that the czar was + /// was restarted when this value changes. + static uint64_t const czarStartupTime; + private: /// Private constructor for singleton. Czar(std::string const& configFilePath, std::string const& czarName); @@ -177,10 +203,12 @@ class Czar { IdToQuery _idToQuery; ///< maps query ID to query (for currently running queries) std::mutex _mutex; ///< protects _uqFactory, _clientToQuery, and _idToQuery - /// Thread pool for handling Responses from XrdSsi, - /// the PsuedoFifo to prevent czar from calling most recent requests, - /// and any other resources for use by query executives. - qdisp::SharedResources::Ptr _qdispSharedResources; + util::Timer _lastRemovedTimer; ///< Timer to limit table deletions. + std::mutex _lastRemovedMtx; ///< protects _lastRemovedTimer + + /// Prevents multiple concurrent calls to _removeOldTables(). + std::atomic _removingOldTables{false}; + std::thread _oldTableRemovalThread; ///< thread needs to remain valid while running. bool _queryDistributionTestVer; ///< True if config says this is distribution test version. @@ -196,7 +224,7 @@ class Czar { /// Connection to the registry to register the czar and get worker contact information. std::shared_ptr _czarRegistry; - std::mutex _executiveMapMtx; ///< protects _executiveMap + mutable std::mutex _executiveMapMtx; ///< protects _executiveMap std::map> _executiveMap; ///< Map of executives for queries in progress. @@ -204,8 +232,28 @@ class Czar { /// Set to false on system shutdown to stop _monitorThrd. std::atomic _monitorLoop{true}; - std::chrono::milliseconds _monitorSleepTime{ - 15000}; ///< Wait time between checks. TODO:UJ set from config + + /// Wait time between checks to. + std::chrono::milliseconds _monitorSleepTime; + + /// Keeps track of all workers (alive or otherwise) that this czar + /// may communicate with. Once created, the pointer never changes. + std::shared_ptr _activeWorkerMap; + + /// A combined priority queue and thread pool to regulate czar communications + /// with workers. Once created, the pointer never changes. + /// TODO:UJ - It may be better to have a pool for each worker as it + /// may be possible for a worker to have communications + /// problems in a way that would wedge the pool. This can + /// probably be done fairly easily by having pools + /// attached to ActiveWorker in _activeWorkerMap. + /// This was not possible in xrootd as the czar had + /// no reasonable way to know where Jobs were going. + std::shared_ptr _qdispPool; + + /// Pool of http client connections for sending commands (UberJobs + /// and worker status requests). + std::shared_ptr _commandHttpPool; }; } // namespace lsst::qserv::czar diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 2570b3dc9a..82d8fd1e8c 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -35,6 +35,8 @@ #include "czar/CzarRegistry.h" #include "qmeta/Exceptions.h" #include "util/Bug.h" +#include "util/InstanceCount.h" //&&& +#include "util/Histogram.h" //&&& #include "util/TimeUtils.h" using namespace std; @@ -84,20 +86,22 @@ void CzarChunkMap::verify() { for (auto const& [chunkId, chunkDataPtr] : chunkMap) { if (chunkDataPtr == nullptr) { - LOGS(_log, LOG_LVL_ERROR, " chunkId=" << chunkId << " had nullptr"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " chunkId=" << chunkId << " had nullptr"); ++errorCount; continue; } auto primeScanWkr = chunkDataPtr->_primaryScanWorker.lock(); if (primeScanWkr == nullptr) { - LOGS(_log, LOG_LVL_ERROR, " chunkId=" << chunkId << " missing primaryScanWorker"); + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " chunkId=" << chunkId << " missing primaryScanWorker"); ++errorCount; continue; } if (primeScanWkr->_sharedScanChunkMap.find(chunkId) == primeScanWkr->_sharedScanChunkMap.end()) { LOGS(_log, LOG_LVL_ERROR, - " chunkId=" << chunkId << " should have been (and was not) in the sharedScanChunkMap for " - << primeScanWkr->_workerId); + cName(__func__) << " chunkId=" << chunkId + << " should have been (and was not) in the sharedScanChunkMap for " + << primeScanWkr->_workerId); ++errorCount; continue; } @@ -105,7 +109,8 @@ void CzarChunkMap::verify() { if (iter != allChunkIds.end()) { allChunkIds.erase(iter); } else { - LOGS(_log, LOG_LVL_ERROR, " chunkId=" << chunkId << " chunkId was not in allChunks list"); + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " chunkId=" << chunkId << " chunkId was not in allChunks list"); ++errorCount; continue; } @@ -118,14 +123,14 @@ void CzarChunkMap::verify() { allMissingIds += to_string(cId) + ","; } LOGS(_log, LOG_LVL_ERROR, - " There were " << missing << " missing chunks from the scan list " << allMissingIds); + cName(__func__) << " There were " << missing << " missing chunks from the scan list " + << allMissingIds); ++errorCount; } if (errorCount > 0) { - // TODO:UJ There may be an argument to keep the new maps even if there are problems - // with them. For current testing, it's probably best to leave it how it is so that - // it's easier to isolate problems. + // Original creation of the family map will keep re-reading until there are no problems. + // _monitor will log this and keep using the old maps. throw ChunkMapException(ERR_LOC, "verification failed with " + to_string(errorCount) + " errors"); } } @@ -161,20 +166,21 @@ void CzarChunkMap::ChunkData::_calcTotalBytes() { void CzarChunkMap::ChunkData::addToWorkerHasThis(std::shared_ptr const& worker) { if (worker == nullptr) { - throw ChunkMapException(ERR_LOC, string(__func__) + " worker was null"); + throw ChunkMapException(ERR_LOC, cName(__func__) + " worker was null"); } _workerHasThisMap[worker->_workerId] = worker; } -std::map> -CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() const { - std::map> newMap = _workerHasThisMap; +map> CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() + const { + map> newMap = _workerHasThisMap; return newMap; } -void CzarChunkMap::organize() { +shared_ptr CzarChunkMap::organize() { auto chunksSortedBySize = make_shared(); + auto missingChunks = make_shared(); calcChunkMap(*_chunkMap, *chunksSortedBySize); @@ -182,36 +188,45 @@ void CzarChunkMap::organize() { // - _workerChunkMap has a map of workerData by worker id with each worker having a map of ChunkData // - _chunkMap has a map of all chunkData by chunk id // - chunksSortedBySize a list of chunks sorted with largest first. - // From here need to assign shared scan chunk priority - // Go through the chunksSortedBySize list and assign each chunk to worker that has it with the smallest - // totalScanSize. + // From here need to assign shared scan chunk priority (i.e. the worker + // that will handle the chunk in shared scans, unless it is dead.) + // Go through the chunksSortedBySize list and assign each chunk to worker that has both: + // - a copy of the chunk + // - the worker currently has the smallest totalScanSize. + // When this is done, all workers should have lists of chunks with similar total sizes + // and missing chunks should be empty. for (auto&& chunkData : *chunksSortedBySize) { SizeT smallest = std::numeric_limits::max(); WorkerChunksData::Ptr smallestWkr = nullptr; + // Find worker with smallest total size. for (auto&& [wkrId, wkrDataWeak] : chunkData->_workerHasThisMap) { auto wkrData = wkrDataWeak.lock(); if (wkrData == nullptr) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " unexpected null weak ptr for " << wkrId); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " unexpected null weak ptr for " << wkrId); continue; // maybe the next one will be okay. } + LOGS(_log, LOG_LVL_DEBUG, - __func__ << " wkrId=" << wkrData << " tsz=" << wkrData->_sharedScanTotalSize - << " smallest=" << smallest); + cName(__func__) << " wkrId=" << wkrData << " tsz=" << wkrData->_sharedScanTotalSize + << " smallest=" << smallest); if (wkrData->_sharedScanTotalSize < smallest) { smallestWkr = wkrData; smallest = smallestWkr->_sharedScanTotalSize; } } if (smallestWkr == nullptr) { - throw ChunkMapException(ERR_LOC, string(__func__) + " no smallesWkr found for chunk=" + - to_string(chunkData->_chunkId)); + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) + " no smallesWkr found for chunk=" + to_string(chunkData->_chunkId)); + missingChunks->push_back(chunkData); + } else { + smallestWkr->_sharedScanChunkMap[chunkData->_chunkId] = chunkData; + smallestWkr->_sharedScanTotalSize += chunkData->_totalBytes; + chunkData->_primaryScanWorker = smallestWkr; + LOGS(_log, LOG_LVL_DEBUG, + " chunk=" << chunkData->_chunkId << " assigned to scan on " << smallestWkr->_workerId); } - smallestWkr->_sharedScanChunkMap[chunkData->_chunkId] = chunkData; - smallestWkr->_sharedScanTotalSize += chunkData->_totalBytes; - chunkData->_primaryScanWorker = smallestWkr; - LOGS(_log, LOG_LVL_DEBUG, - " chunk=" << chunkData->_chunkId << " assigned to scan on " << smallestWkr->_workerId); } + return missingChunks; } string CzarChunkMap::ChunkData::dump() const { @@ -231,6 +246,34 @@ string CzarChunkMap::ChunkData::dump() const { return os.str(); } +bool CzarChunkMap::WorkerChunksData::isDead() { + if (_activeWorker == nullptr) { + // At startup, these may not be available + auto czarPtr = Czar::getCzar(); + if (czarPtr == nullptr) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " czarPtr is null, this should only happen in unit test."); + return false; + } + auto awMap = Czar::getCzar()->getActiveWorkerMap(); + if (awMap == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " awMap is null."); + return true; + } + _activeWorker = awMap->getActiveWorker(_workerId); + if (_activeWorker == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " activeWorker not found."); + return true; + } + } + auto wState = _activeWorker->getState(); + bool dead = wState == ActiveWorker::DEAD; + if (dead) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " is dead"); + } + return dead; +} + string CzarChunkMap::WorkerChunksData::dump() const { stringstream os; os << "{WorkerChunksData id=" << _workerId << " scanTotalSize=" << _sharedScanTotalSize; @@ -246,11 +289,29 @@ string CzarChunkMap::WorkerChunksData::dump() const { return os.str(); } +CzarFamilyMap::Ptr CzarFamilyMap::create(std::shared_ptr const& qmeta) { + // There's nothing the czar can do until with user queries until there's been at least + // one successful read of the database family tables, as the czar doesn't know where to find anything. + Ptr newPtr = nullptr; + while (newPtr == nullptr) { + try { + newPtr = Ptr(new CzarFamilyMap(qmeta)); + } catch (ChunkMapException const& exc) { + LOGS(_log, LOG_LVL_WARN, "Could not create CzarFamilyMap, sleep and retry " << exc.what()); + } + if (newPtr == nullptr) { + this_thread::sleep_for(10s); + } + } + + return newPtr; +} + CzarFamilyMap::CzarFamilyMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { try { auto mapsSet = _read(); if (!mapsSet) { - throw ChunkMapException(ERR_LOC, cName(__func__) + " maps were not set in contructor"); + throw ChunkMapException(ERR_LOC, cName(__func__) + " maps were not set in constructor"); } } catch (qmeta::QMetaError const& qExc) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not read DB " << qExc.what()); @@ -272,9 +333,9 @@ bool CzarFamilyMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarFamilyMap::_read() start"); // If replacing the map, this may take a bit of time, but it's probably // better to wait for new maps if something changed. - std::lock_guard gLock(_familyMapMtx); + std::lock_guard gLock(_familyMapMtx); // &&& check waiting is really needed qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(_lastUpdateTime); - if (_lastUpdateTime >= qChunkMap.updateTime) { + if (_lastUpdateTime == qChunkMap.updateTime) { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " no need to read " << util::TimeUtils::timePointToDateTimeString(_lastUpdateTime) @@ -295,9 +356,13 @@ bool CzarFamilyMap::_read() { return true; } +util::HistogramRolling histoMakeNewMaps("&&&uj histoMakeNewMaps", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + std::shared_ptr CzarFamilyMap::makeNewMaps( qmeta::QMetaChunkMap const& qChunkMap) { // Create new maps. + util::InstanceCount ic("CzarFamilyMap::makeNewMaps&&&"); + auto startMakeMaps = CLOCK::now(); //&&& std::shared_ptr newFamilyMap = make_shared(); // Workers -> Databases map @@ -332,12 +397,29 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( } } - // this needs to be done for each CzarChunkMap in the family map. + // This needs to be done for each CzarChunkMap in the family map. for (auto&& [familyName, chunkMapPtr] : *newFamilyMap) { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " working on " << familyName); - chunkMapPtr->organize(); + auto missing = chunkMapPtr->organize(); + if (missing != nullptr && !missing->empty()) { + // TODO:UJ Some element of the dashboard should be made aware of this. Also, + // TODO:UJ maybe this should check all families before throwing. + // TODO:UJ There are implications that maybe the replicator should not + // TODO:UJ tell the czar about families/databases that do not have + // TODO:UJ at least one copy of each chunk with data loaded on a worker. + string chunkIdStr; + for (auto const& chunkData : *missing) { + chunkIdStr += to_string(chunkData->getChunkId()) + " "; + } + throw ChunkMapException( + ERR_LOC, cName(__func__) + " family=" + familyName + " is missing chunks " + chunkIdStr); + } } + auto endMakeMaps = CLOCK::now(); //&&& + std::chrono::duration secsMakeMaps = endMakeMaps - startMakeMaps; // &&& + histoMakeNewMaps.addEntry(endMakeMaps, secsMakeMaps.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoMakeNewMaps.getString("")); return newFamilyMap; } diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index b6f69d38b2..28bc023192 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -43,6 +43,7 @@ struct QMetaChunkMap; namespace lsst::qserv::czar { +class ActiveWorker; class CzarFamilyMap; class ChunkMapException : public util::Issue { @@ -71,10 +72,11 @@ class CzarChunkMap { using Ptr = std::shared_ptr; using SizeT = uint64_t; + std::string cName(const char* func) { return std::string("CzarChunkMap::") + func; } + CzarChunkMap(CzarChunkMap const&) = delete; CzarChunkMap& operator=(CzarChunkMap const&) = delete; - // static Ptr create(std::shared_ptr const& qmeta) { return Ptr(new CzarChunkMap(qmeta)); } static Ptr create() { return Ptr(new CzarChunkMap()); } ~CzarChunkMap(); @@ -88,8 +90,10 @@ class CzarChunkMap { using Ptr = std::shared_ptr; ChunkData(int chunkId_) : _chunkId(chunkId_) {} + std::string cName(const char* func) { + return std::string("ChunkData::") + func + " " + std::to_string(_chunkId); + } int64_t getChunkId() const { return _chunkId; } - SizeT getTotalBytes() const { return _totalBytes; } std::weak_ptr getPrimaryScanWorker() const { return _primaryScanWorker; } @@ -127,6 +131,10 @@ class CzarChunkMap { using Ptr = std::shared_ptr; WorkerChunksData(std::string const& workerId) : _workerId(workerId) {} + std::string cName(const char* func) { + return std::string("WorkerChunksData::") + func + " " + _workerId; + } + /// Return the worker's id string. std::string const& getWorkerId() const { return _workerId; } @@ -134,6 +142,9 @@ class CzarChunkMap { /// accessed in a full table scan on this worker. SizeT getSharedScanTotalSize() const { return _sharedScanTotalSize; } + /// Return true if this worker is dead, according to `ActiveWorkerMap`. + bool isDead(); + /// Return a reference to `_sharedScanChunkMap`. A copy of the pointer /// to this class (or the containing map) should be held to ensure the reference. std::map const& getSharedScanChunkMap() const { return _sharedScanChunkMap; } @@ -152,13 +163,17 @@ class CzarChunkMap { /// Map of chunks this worker will handle during shared scans. /// Since scans are done in order of chunk id numbers, it helps /// to have this in chunk id number order. - /// At some point, thus should be sent to workers so they + /// At some point, this should be sent to workers so they /// can make more accurate time estimates for chunk completion. std::map _sharedScanChunkMap; /// The total size (in bytes) of all chunks on this worker that /// are to be used in shared scans. SizeT _sharedScanTotalSize = 0; + + /// Used to determine if this worker is alive and set + /// when the test is made. + std::shared_ptr _activeWorker; }; using WorkerChunkMap = std::map; @@ -191,8 +206,11 @@ class CzarChunkMap { } /// Use the information from the registry to `organize` `_chunkMap` and `_workerChunkMap` - /// into their expected formats. - void organize(); + /// into their expected formats, which also should define where a chunk is always + /// run during shared scans. + /// This is a critical function for defining which workers will handle which jobs. + /// @return a vector of ChunkData::Ptr of chunks where no worker was found. + std::shared_ptr organize(); private: CzarChunkMap(); @@ -244,7 +262,7 @@ class CzarFamilyMap { typedef std::map FamilyMapType; typedef std::map DbNameToFamilyNameType; - static Ptr create(std::shared_ptr const& qmeta) { return Ptr(new CzarFamilyMap(qmeta)); } + static Ptr create(std::shared_ptr const& qmeta); CzarFamilyMap() = delete; CzarFamilyMap(CzarFamilyMap const&) = delete; @@ -280,6 +298,9 @@ class CzarFamilyMap { /// Make a new FamilyMapType map including ChunkMap and WorkerChunkMap from the data /// in `qChunkMap`. Each family has its own ChunkMap and WorkerChunkMap. + /// + /// NOTE: This is likely an expensive operation and should probably only + /// be called if new workers have been added or chunks have been moved. std::shared_ptr makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); /// Insert the new element described by the parameters into the `newFamilyMap` as appropriate. diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index f5abfcaba6..e81b0e168c 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -48,7 +48,9 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarRegistry"); namespace lsst::qserv::czar { -CzarRegistry::CzarRegistry(std::shared_ptr const& czarConfig) : _czarConfig(czarConfig) { +CzarRegistry::CzarRegistry(cconfig::CzarConfig::Ptr const& czarConfig, + ActiveWorkerMap::Ptr const& activeWorkerMap) + : _czarConfig(czarConfig), _activeWorkerMap(activeWorkerMap) { // Begin periodically updating worker's status in the Replication System's registry. // This will continue until the application gets terminated. thread registryUpdateThread(&CzarRegistry::_registryUpdateLoop, this); @@ -68,6 +70,11 @@ CzarRegistry::~CzarRegistry() { } } +protojson::WorkerContactInfo::WCMapPtr CzarRegistry::getWorkerContactMap() const { + lock_guard lockG(_cmapMtx); + return _contactMap; +} + void CzarRegistry::_registryUpdateLoop() { auto const method = http::Method::POST; string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + @@ -103,6 +110,10 @@ void CzarRegistry::_registryUpdateLoop() { void CzarRegistry::_registryWorkerInfoLoop() { // Get worker information from the registry + string const replicationInstanceId = _czarConfig->replicationInstanceId(); + string const replicationAuthKey = _czarConfig->replicationAuthKey(); + uint64_t const czarStartTime = Czar::czarStartupTime; + vector const headers; auto const method = http::Method::GET; string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + @@ -119,13 +130,18 @@ void CzarRegistry::_registryWorkerInfoLoop() { LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); // TODO: Is there a better thing to do than just log this here? } else { - WorkerContactMapPtr wMap = _buildMapFromJson(response); - // Compare the new map to the existing map and replace if different. + protojson::WorkerContactInfo::WCMapPtr wMap = _buildMapFromJson(response); + // Update the values in the map { - lock_guard lck(_mapMtx); - if (wMap != nullptr && !_compareMap(*wMap)) { + auto czInfo = protojson::CzarContactInfo::create( + _czarConfig->name(), _czarConfig->id(), _czarConfig->replicationHttpPort(), + util::get_current_host_fqdn(), czarStartTime); + lock_guard lck(_cmapMtx); + if (wMap != nullptr) { _contactMap = wMap; - _latestUpdate = CLOCK::now(); + _latestMapUpdate = CLOCK::now(); + _activeWorkerMap->updateMap(*_contactMap, czInfo, replicationInstanceId, + replicationAuthKey); } } } @@ -137,25 +153,22 @@ void CzarRegistry::_registryWorkerInfoLoop() { } } -CzarRegistry::WorkerContactMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { +protojson::WorkerContactInfo::WCMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { auto const& jsServices = response.at("services"); auto const& jsWorkers = jsServices.at("workers"); - auto wMap = WorkerContactMapPtr(new WorkerContactMap()); + auto wMap = protojson::WorkerContactInfo::WCMapPtr(new protojson::WorkerContactInfo::WCMap()); for (auto const& [key, value] : jsWorkers.items()) { auto const& jsQserv = value.at("qserv"); LOGS(_log, LOG_LVL_DEBUG, __func__ << " key=" << key << " jsQ=" << jsQserv); - string wHost = jsQserv.at("host-addr").get(); - string wManagementHost = jsQserv.at("management-host-name").get(); - int wPort = jsQserv.at("management-port").get(); - uint64_t updateTimeInt = jsQserv.at("update-time-ms").get(); - TIMEPOINT updateTime = TIMEPOINT(chrono::milliseconds(updateTimeInt)); - auto wInfo = make_shared(key, wHost, wManagementHost, wPort, updateTime); - LOGS(_log, LOG_LVL_DEBUG, - __func__ << " wHost=" << wHost << " wPort=" << wPort << " updateTime=" << updateTimeInt); + + // The names for items here are different than the names used by workers. + auto wInfo = protojson::WorkerContactInfo::createFromJsonRegistry(key, jsQserv); + + LOGS(_log, LOG_LVL_DEBUG, __func__ << " wInfot=" << wInfo->dump()); auto iter = wMap->find(key); if (iter != wMap->end()) { LOGS(_log, LOG_LVL_ERROR, __func__ << " duplicate key " << key << " in " << response); - if (!wInfo->sameContactInfo(*(iter->second))) { + if (!wInfo->isSameContactInfo(*(iter->second))) { LOGS(_log, LOG_LVL_ERROR, __func__ << " incongruent key " << key << " in " << response); return nullptr; } @@ -167,7 +180,8 @@ CzarRegistry::WorkerContactMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json return wMap; } -bool CzarRegistry::_compareMap(WorkerContactMap const& other) const { +bool CzarRegistry::_compareMapContactInfo(protojson::WorkerContactInfo::WCMap const& other) const { + VMUTEX_HELD(_cmapMtx); if (_contactMap == nullptr) { // If _contactMap is null, it needs to be replaced. return false; @@ -180,7 +194,7 @@ bool CzarRegistry::_compareMap(WorkerContactMap const& other) const { if (iter == other.end()) { return false; } else { - if (!(iter->second->sameContactInfo(*wInfo))) { + if (!(iter->second->isSameContactInfo(*wInfo))) { return false; } } @@ -188,11 +202,37 @@ bool CzarRegistry::_compareMap(WorkerContactMap const& other) const { return true; } -string CzarRegistry::WorkerContactInfo::dump() const { - stringstream os; - os << "workerContactInfo{" - << "id=" << wId << " host=" << wHost << " mgHost=" << wManagementHost << " port=" << wPort << "}"; - return os.str(); +protojson::WorkerContactInfo::WCMapPtr CzarRegistry::waitForWorkerContactMap() const { + protojson::WorkerContactInfo::WCMapPtr contMap = nullptr; + while (contMap == nullptr) { + { + lock_guard lockG(_cmapMtx); + contMap = _contactMap; + } + if (contMap == nullptr) { + // This should only ever happen at startup if there's trouble getting data. + LOGS(_log, LOG_LVL_WARN, "waitForWorkerContactMap() _contactMap unavailable waiting for info"); + this_thread::sleep_for(1s); + } + } + return contMap; +} + +void CzarRegistry::sendActiveWorkersMessages() { + // Send messages to each active worker as needed + _activeWorkerMap->sendActiveWorkersMessages(); +} + +void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) { + // Add query id to the appropriate list. + if (deleteWorkerResults) { + _activeWorkerMap->addToDoneDeleteFiles(qId); + } else { + _activeWorkerMap->addToDoneKeepFiles(qId); + } + + // With lists updated, send out messages. + _activeWorkerMap->sendActiveWorkersMessages(); } } // namespace lsst::qserv::czar diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index 27d20979cf..08d24a7bcc 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -34,7 +34,9 @@ #include "nlohmann/json.hpp" // Qserv headers +#include "czar/ActiveWorker.h" #include "global/clock_defs.h" +#include "util/Mutex.h" namespace lsst::qserv::cconfig { class CzarConfig; @@ -60,49 +62,36 @@ class CzarRegistry { using Ptr = std::shared_ptr; /// Return a pointer to a new CzarRegistry object. - static Ptr create(std::shared_ptr const& czarConfig) { - return Ptr(new CzarRegistry(czarConfig)); + static Ptr create(std::shared_ptr const& czarConfig, + std::shared_ptr const& activeWorkerMap) { + return Ptr(new CzarRegistry(czarConfig, activeWorkerMap)); } ~CzarRegistry(); - struct WorkerContactInfo { - using Ptr = std::shared_ptr; - - WorkerContactInfo(std::string const& wId_, std::string const& wHost_, - std::string const& wManagementHost_, int wPort_, TIMEPOINT updateTime_) - : wId(wId_), - wHost(wHost_), - wManagementHost(wManagementHost_), - wPort(wPort_), - updateTime(updateTime_) {} - std::string const wId; ///< key - std::string const wHost; ///< "host-addr" entry. - std::string const wManagementHost; ///< "management-host-name" entry. - int const wPort; ///< "management-port" entry. - TIMEPOINT const updateTime; ///< "update-time-ms" entry. - - /// Return true if all members, aside from updateTime, are equal. - bool sameContactInfo(WorkerContactInfo const& other) const { - return (wId == other.wId && wHost == other.wHost && wManagementHost == other.wManagementHost && - wPort == other.wPort); - } - std::string dump() const; - }; - - using WorkerContactMap = std::unordered_map; - using WorkerContactMapPtr = std::shared_ptr; - /// Return _contactMap, the object that the returned pointer points to is /// constant and no attempts should be made to change it. - WorkerContactMapPtr getWorkerContactMap() { - std::lock_guard lockG(_mapMtx); - return _contactMap; - } + protojson::WorkerContactInfo::WCMapPtr getWorkerContactMap() const; + + /// Return _contactMap, the object that the returned pointer points to is + /// constant and no attempts should be made to change it. This + /// function will wait forever for a valid contact map to be ready. + protojson::WorkerContactInfo::WCMapPtr waitForWorkerContactMap() const; + + /// Send all live workers the `WorkerQueryStatusData` message for + /// that worker. This may result in the worker sending back the + /// `WorkerCzarComIssue` message if there were communication problems. + void sendActiveWorkersMessages(); + + /// Add the query id to the list of queries to end on workers and + /// send the messages, deleting all result files if + /// `deleteWorkerResults` is true. + void endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults); private: CzarRegistry() = delete; - CzarRegistry(std::shared_ptr const& czarConfig); + CzarRegistry(std::shared_ptr const& czarConfig, + std::shared_ptr const& activeWorkerMap); /// This function will keep periodically updating Czar's info in the Replication System's Registry /// until _loop is set to false. @@ -115,10 +104,11 @@ class CzarRegistry { void _registryWorkerInfoLoop(); /// Build a new WorkerContactMap from the json `response` - WorkerContactMapPtr _buildMapFromJson(nlohmann::json const& response); + protojson::WorkerContactInfo::WCMapPtr _buildMapFromJson(nlohmann::json const& response); - /// Return true if maps are the same size and all of the elements are the same(). - bool _compareMap(WorkerContactMap const& other) const; + /// Return true if maps are the same size and all of the elements have the same contact info. + /// NOTE: _cmapMtx must be held when calling. + bool _compareMapContactInfo(protojson::WorkerContactInfo::WCMap const& other) const; std::shared_ptr const _czarConfig; ///< Pointer to the CzarConfig. @@ -127,9 +117,13 @@ class CzarRegistry { std::thread _czarWorkerInfoThrd; ///< This thread continuously collects worker contact information. /// Pointer to the map of worker contact information. - WorkerContactMapPtr _contactMap; - TIMEPOINT _latestUpdate; ///< The last time the _contactMap was updated. - std::mutex _mapMtx; /// Protects _contactMap, _latestUpdate. + protojson::WorkerContactInfo::WCMapPtr _contactMap; + TIMEPOINT _latestMapUpdate; ///< The last time the _contactMap was updated, unrelated to + ///< WorkerContactInfo update. + mutable MUTEX _cmapMtx; /// Protects _contactMap, _latestUpdate + + /// Map for tracking worker aliveness, it has its own internal mutex. + std::shared_ptr const _activeWorkerMap; }; } // namespace lsst::qserv::czar diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 3d179d577f..266fdbdbe9 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -57,16 +57,19 @@ void HttpCzarWorkerModule::process(string const& context, shared_ptr const& req, shared_ptr const& resp) - : HttpModule(context, req, resp) {} + : QhttpModule(context, req, resp) {} json HttpCzarWorkerModule::executeImpl(string const& subModuleName) { string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; debug(func); + cconfig::CzarConfig::instance()->replicationInstanceId(); enforceCzarName(func); if (subModuleName == "QUERYJOB-ERROR") return _queryJobError(); else if (subModuleName == "QUERYJOB-READY") return _queryJobReady(); + else if (subModuleName == "WORKERCZARCOMISSUE") + return _workerCzarComIssue(); throw invalid_argument(context() + func + " unsupported sub-module"); } @@ -86,13 +89,22 @@ json HttpCzarWorkerModule::_queryJobReady() { return ret; } +json HttpCzarWorkerModule::_workerCzarComIssue() { + debug(__func__); + checkApiVersion(__func__, 34); + LOGS(_log, LOG_LVL_DEBUG, __func__ << " workerczarcomissue json=" << body().objJson); + auto ret = _handleWorkerCzarComIssue(__func__); + return ret; +} + json HttpCzarWorkerModule::_handleJobError(string const& func) { + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobError start"); // Metadata-only responses for the file-based protocol should not have any data // Parse and verify the json message and then kill the UberJob. json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // See qdisp::UberJob::runUberJob() for json message construction. + // See qdisp::UberJob::runUberJob() for json message construction. &&& string const targetWorkerId = body().required("workerid"); string const czarName = body().required("czar"); qmeta::CzarId const czarId = body().required("czarid"); @@ -122,16 +134,20 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { "HttpCzarWorkerModule::_handleJobError received " << iaEx.what() << " js=" << body().objJson); jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; } + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobError end"); return jsRet; } json HttpCzarWorkerModule::_handleJobReady(string const& func) { + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady start"); // Metadata-only responses for the file-based protocol should not have any data // Parse and verify the json message and then have the uberjob import the file. json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // See qdisp::UberJob::runUberJob() for json message construction. + // &&& TODO:UJ file response - move construction and parsing + // &&& TODO:UJ to a class so it can be added to WorkerCzarComIssue + // See qdisp::UberJob::runUberJob() for json message construction. &&& string const targetWorkerId = body().required("workerid"); string const czarName = body().required("czar"); qmeta::CzarId const czarId = body().required("czarid"); @@ -162,6 +178,45 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { "HttpCzarWorkerModule::_handleJobReady received " << iaEx.what() << " js=" << body().objJson); jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; } + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady end"); + return jsRet; +} + +json HttpCzarWorkerModule::_handleWorkerCzarComIssue(string const& func) { + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleWorkerCzarComIssue start"); + // Parse and verify the json message and then deal with the problems. + json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + string const replicationInstanceId = cconfig::CzarConfig::instance()->replicationInstanceId(); + string const replicationAuthKey = cconfig::CzarConfig::instance()->replicationAuthKey(); + auto const& jsReq = body().objJson; + auto wccIssue = protojson::WorkerCzarComIssue::createFromJson(jsReq, replicationInstanceId, + replicationAuthKey); + + auto wId = wccIssue->getWorkerInfo()->wId; + if (wccIssue->getThoughtCzarWasDead()) { + LOGS(_log, LOG_LVL_WARN, + "HttpCzarWorkerModule::_handleWorkerCzarComIssue worker=" + << wId << " thought czar was dead and killed related uberjobs."); + + // Find all incomplete UberJobs with this workerId and re-assign them. + // Use a copy to avoid mutex issues. + auto execMap = czar::Czar::getCzar()->getExecMapCopy(); + for (auto const& [exKey, execWeak] : execMap) { + auto execPtr = execWeak.lock(); + if (execPtr == nullptr) continue; + execPtr->killIncompleteUberJobsOnWorker(wId); + } + } + jsRet = wccIssue->serializeResponseJson(); + LOGS(_log, LOG_LVL_TRACE, "HttpCzarWorkerModule::_handleWorkerCzarComIssue jsRet=" << jsRet.dump()); + + } catch (std::invalid_argument const& iaEx) { + LOGS(_log, LOG_LVL_ERROR, + "HttpCzarWorkerModule::_handleWorkerCzarComIssue received " << iaEx.what() + << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; + } return jsRet; } diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h index ef0a745093..a6d21536c0 100644 --- a/src/czar/HttpCzarWorkerModule.h +++ b/src/czar/HttpCzarWorkerModule.h @@ -29,7 +29,7 @@ #include "nlohmann/json.hpp" // Qserv headers -#include "czar/HttpModule.h" +#include "czar/QhttpModule.h" // Forward declarations namespace lsst::qserv::qhttp { @@ -41,7 +41,7 @@ class Response; namespace lsst::qserv::czar { /// This class is used to handle messages to this czar from the workers. -class HttpCzarWorkerModule : public czar::HttpModule { +class HttpCzarWorkerModule : public QhttpModule { public: /// @note supported values for parameter 'subModuleName' are: /// 'QUERYJOB-ERROR' - error in a QUERYJOB @@ -70,11 +70,17 @@ class HttpCzarWorkerModule : public czar::HttpModule { /// Called to indicate an UberJob is ready with data that needs to be collected. nlohmann::json _queryJobReady(); + /// Called to indicate there were problems with the worker trying to reach this czar. + nlohmann::json _workerCzarComIssue(); + /// Translates the message and calls the Czar to collect the data. nlohmann::json _handleJobReady(std::string const& func); /// Translates the error and calls the Czar to take action. nlohmann::json _handleJobError(std::string const& func); + + /// Translates the issues and calls the Czar to take action. + nlohmann::json _handleWorkerCzarComIssue(std::string const& func); }; } // namespace lsst::qserv::czar diff --git a/src/czar/HttpSvc.cc b/src/czar/HttpSvc.cc index b67330e27d..3d953cdab8 100644 --- a/src/czar/HttpSvc.cc +++ b/src/czar/HttpSvc.cc @@ -101,6 +101,11 @@ uint16_t HttpSvc::start() { [self](shared_ptr const& req, shared_ptr const& resp) { HttpCzarWorkerModule::process(::serviceName, req, resp, "QUERYJOB-READY"); }}}); + _httpServerPtr->addHandlers( + {{"POST", "/workerczarcomissue", + [self](shared_ptr const& req, shared_ptr const& resp) { + HttpCzarWorkerModule::process(::serviceName, req, resp, "WORKERCZARCOMISSUE"); + }}}); _httpServerPtr->start(); // Initialize the I/O context and start the service threads. At this point diff --git a/src/global/CMakeLists.txt b/src/global/CMakeLists.txt index f0bf7ad498..25e94c496d 100644 --- a/src/global/CMakeLists.txt +++ b/src/global/CMakeLists.txt @@ -28,6 +28,5 @@ FUNCTION(global_tests) ENDFUNCTION() global_tests( - testResourceUnit testStringUtil ) diff --git a/src/global/ResourceUnit.cc b/src/global/ResourceUnit.cc index 64144b8436..3bbe5372bd 100644 --- a/src/global/ResourceUnit.cc +++ b/src/global/ResourceUnit.cc @@ -31,43 +31,6 @@ namespace lsst::qserv { -////////////////////////////////////////////////////////////////////// -// lsst::qserv::ResourceUnit::Tokenizer -// A simple class to tokenize paths. -////////////////////////////////////////////////////////////////////// -class ResourceUnit::Tokenizer { -public: - Tokenizer(std::string const& s, char sep = '/') : _cursor(0), _next(0), _s(s), _sep(sep) { _seek(); } - - std::string token() { return _s.substr(_cursor, _next - _cursor); } - - int tokenAsInt() { - int num; - std::istringstream csm(token()); - csm >> num; - return num; - } - - void next() { - assert(!done()); - _cursor = _next + 1; - _seek(); - } - - bool done() { return _next == std::string::npos; } - -private: - void _seek() { _next = _s.find_first_of(_sep, _cursor); } - - std::string::size_type _cursor; - std::string::size_type _next; - std::string const _s; - char const _sep; -}; - -////////////////////////////////////////////////////////////////////// -ResourceUnit::ResourceUnit(std::string const& path) : _unitType(GARBAGE), _chunk(-1) { _setFromPath(path); } - std::string ResourceUnit::path() const { std::stringstream ss; ss << _pathSep << prefix(_unitType); @@ -90,14 +53,6 @@ std::string ResourceUnit::path() const { return ss.str(); } -std::string ResourceUnit::var(std::string const& key) const { - VarMap::const_iterator ci = _vars.find(key); - if (ci != _vars.end()) { - return ci->second; - } - return std::string(); -} - std::string ResourceUnit::prefix(UnitType const& r) { switch (r) { case DBCHUNK: @@ -122,88 +77,6 @@ void ResourceUnit::setAsDbChunk(std::string const& db, int chunk) { _chunk = chunk; } -bool ResourceUnit::_markGarbageIfDone(Tokenizer& t) { - if (t.done()) { - _unitType = GARBAGE; - return true; - } - return false; -} - -void ResourceUnit::_setFromPath(std::string const& path) { - std::string rTypeString; - Tokenizer t(path, _pathSep); - if (!t.token().empty()) { // Expect leading separator (should start with /) - _unitType = UNKNOWN; - return; - } - if (_markGarbageIfDone(t)) { - return; - } // Consider using GOTO structure. - t.next(); - rTypeString = t.token(); - if (rTypeString == prefix(DBCHUNK)) { - // XrdSsi query - if (_markGarbageIfDone(t)) { - return; - } - _unitType = DBCHUNK; - t.next(); - _db = t.token(); - if (_db.empty()) { - _unitType = GARBAGE; - return; - } - if (_markGarbageIfDone(t)) { - return; - } - t.next(); - if (t.token().empty()) { - _unitType = GARBAGE; - return; - } - _chunk = t.tokenAsInt(); - _ingestLeafAndKeys(t.token()); - } else if (rTypeString == prefix(QUERY)) { - _unitType = QUERY; - if (!t.done()) { - _unitType = GARBAGE; - return; - } - } else { - _unitType = GARBAGE; - } -} - -/// Ingest key-value pairs from a string including the last portion of the path, -/// e.g., somenumber?key1=val1&key2=val2 -void ResourceUnit::_ingestLeafAndKeys(std::string const& leafPlusKeys) { - std::string::size_type start; - start = leafPlusKeys.find_first_of(_varSep, 0); - _vars.clear(); - - if (start == std::string::npos) { // No keys found - return; - } - ++start; - Tokenizer t(leafPlusKeys.substr(start), _varDelim); - for (std::string defn = t.token(); !defn.empty(); t.next()) { - _ingestKeyStr(defn); - } -} - -/// Ingest key-value pairs from a packed key-value representation. -/// e.g., key1=val1&key2=val2 -void ResourceUnit::_ingestKeyStr(std::string const& keyStr) { - std::string::size_type equalsPos; - equalsPos = keyStr.find_first_of('='); - if (equalsPos == std::string::npos) { // No = clause, value-less key. - _vars[keyStr] = std::string(); // empty insert. - } else { - _vars[keyStr.substr(0, equalsPos)] = keyStr.substr(equalsPos + 1); - } -} - std::ostream& operator<<(std::ostream& os, ResourceUnit const& ru) { return os << "Resource(" << ru.path() << ")"; } diff --git a/src/global/ResourceUnit.h b/src/global/ResourceUnit.h index ad4a1ef0be..50cd69b0e9 100644 --- a/src/global/ResourceUnit.h +++ b/src/global/ResourceUnit.h @@ -33,22 +33,13 @@ namespace lsst::qserv { -/// ResourceUnit contains a name for an XrdSsi-resolvable resource unit. -//// -/// Not sure this belongs in global, but czar, worker both need it. -/// Other components may as well. -//// -/// Note that while key-value specifiers are parsed from the path string at -/// construction, the code for generating a path that includes the key-value -/// portion is not implemented. It is unclear whether we need the generation -/// capability, now that key-value pairs can be packed in protobufs messages. +/// This class is used to store the database and chunk id of a resource. class ResourceUnit { public: class Checker; enum UnitType { GARBAGE, DBCHUNK, UNKNOWN, QUERY }; ResourceUnit() = default; - explicit ResourceUnit(std::string const& path); ResourceUnit(ResourceUnit const&) = default; ResourceUnit& operator=(ResourceUnit const&) = default; ~ResourceUnit() = default; @@ -62,9 +53,6 @@ class ResourceUnit { std::string const& db() const { return _db; } int chunk() const { return _chunk; } - /// Lookup extended path variables (?k=val syntax) - std::string var(std::string const& key) const; - /// @return the path prefix element for a given request type. static std::string prefix(UnitType const& r); @@ -75,32 +63,15 @@ class ResourceUnit { void setAsDbChunk(std::string const& db, int chunk = DUMMY_CHUNK); private: - class Tokenizer; - void _setFromPath(std::string const& path); - void _ingestLeafAndKeys(std::string const& leafPlusKeys); - void _ingestKeyStr(std::string const& keyStr); - bool _markGarbageIfDone(Tokenizer& t); - UnitType _unitType = UnitType::GARBAGE; //< Type of unit std::string _db; //< for DBCHUNK type int _chunk = -1; //< for DBCHUNK type - typedef std::map VarMap; - VarMap _vars; //< Key-value specifiers - static char const _pathSep = '/'; - static char const _varSep = '?'; - static char const _varDelim = '&'; friend std::ostream& operator<<(std::ostream& os, ResourceUnit const& ru); }; -class ResourceUnit::Checker { -public: - virtual ~Checker() {} - virtual bool operator()(ResourceUnit const& ru) = 0; -}; - } // namespace lsst::qserv #endif // LSST_QSERV_RESOURCEUNIT_H diff --git a/src/global/clock_defs.h b/src/global/clock_defs.h index 9db4dadbc8..25d3b08bf8 100644 --- a/src/global/clock_defs.h +++ b/src/global/clock_defs.h @@ -37,6 +37,10 @@ namespace lsst::qserv { using CLOCK = std::chrono::system_clock; using TIMEPOINT = std::chrono::time_point; +inline uint64_t millisecSinceEpoch(TIMEPOINT tm) { + return std::chrono::duration_cast(tm.time_since_epoch()).count(); +} + /// RAII class to help track a changing sum through a begin and end time. template class TimeCountTracker { diff --git a/src/global/intTypes.h b/src/global/intTypes.h index c3a6f7fb07..8463644e57 100644 --- a/src/global/intTypes.h +++ b/src/global/intTypes.h @@ -38,7 +38,8 @@ typedef std::vector Int32Vector; /// Typedef for Query ID in query metadata. typedef std::uint64_t QueryId; typedef std::int64_t JobId; -typedef JobId UberJobId; // These must be the same type. +typedef JobId UberJobId; // These must be the same type. +typedef std::uint32_t CzarIdType; // TODO:UJ remove qmeta::CzarId and rename this CzarId /// Class to provide a consistent format for QueryIds in the log file class QueryIdHelper { diff --git a/src/global/testResourceUnit.cc b/src/global/testResourceUnit.cc deleted file mode 100644 index dfde0e3c23..0000000000 --- a/src/global/testResourceUnit.cc +++ /dev/null @@ -1,91 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// testResourceUnit - -// Third-party headers - -// Qserv headers -#include "global/ResourceUnit.h" - -// Boost unit test header -#define BOOST_TEST_MODULE ResourceUnit_1 -#include - -namespace test = boost::test_tools; -using lsst::qserv::ResourceUnit; - -struct Fixture { - Fixture() : dummy(0) {} - - int dummy; - ~Fixture(void) {}; -}; -int const MAGIC_SIZE = 80; - -BOOST_FIXTURE_TEST_SUITE(Suite, Fixture) - -BOOST_AUTO_TEST_CASE(Garbage) { - char p[][MAGIC_SIZE] = {// Convert to std vector list init when available - // Missing chunk number - "/chk/qcase01", "/chk/abc/", - // Bad resource type - "/chk2/abc", "/abc/", "/abc/chk/g", - // Missing/bad params - "/q", "/q/", "/q/Hello", "/result", "/result/"}; - int const pSize = 10; - for (auto i = p, e = p + pSize; i != e; ++i) { - ResourceUnit r(*i); - BOOST_CHECK_MESSAGE(r.unitType() == ResourceUnit::GARBAGE, std::string("Expected garbage: ") + *i); - } -} - -BOOST_AUTO_TEST_CASE(DbChunk) { - char p[][MAGIC_SIZE] = { - "/chk/qcase01/123", - "/chk/abc/456", - }; - int const pSize = 2; - std::vector r; - for (auto i = p, e = p + pSize; i != e; ++i) { - r.push_back(ResourceUnit(*i)); - BOOST_CHECK_EQUAL(r.back().unitType(), ResourceUnit::DBCHUNK); - } - BOOST_CHECK_EQUAL(r[0].db(), "qcase01"); - BOOST_CHECK_EQUAL(r[1].db(), "abc"); - BOOST_CHECK_EQUAL(r[0].chunk(), 123); - BOOST_CHECK_EQUAL(r[1].chunk(), 456); - - r[0].setAsDbChunk("foo", 1111); - r[1].setAsDbChunk("bar", 968); - BOOST_CHECK_EQUAL(r[0].path(), "/chk/foo/1111"); - BOOST_CHECK_EQUAL(r[1].path(), "/chk/bar/968"); -} - -BOOST_AUTO_TEST_CASE(Query) { - ResourceUnit const res1("/query"); - BOOST_CHECK_EQUAL(res1.unitType(), ResourceUnit::QUERY); - ResourceUnit const res2("/query/abc"); - BOOST_CHECK_EQUAL(res2.unitType(), ResourceUnit::GARBAGE); -} - -BOOST_AUTO_TEST_SUITE_END() diff --git a/src/http/BaseModule.h b/src/http/BaseModule.h index 1e615fcbbe..bef586d22e 100644 --- a/src/http/BaseModule.h +++ b/src/http/BaseModule.h @@ -215,6 +215,8 @@ class BaseModule { */ void sendData(nlohmann::json& result); + std::string authKey() const { return _authKey; } + private: /** * Check the authorization keys provided in the request's body against the expected values. diff --git a/src/http/Client.cc b/src/http/Client.cc index 1f4e2c690c..ae713f1f87 100644 --- a/src/http/Client.cc +++ b/src/http/Client.cc @@ -146,7 +146,6 @@ void Client::read(CallbackType const& onDataRead) { } _curlEasyErrorChecked("curl_easy_setopt(CURLOPT_HTTPHEADER)", curl_easy_setopt(_hcurl, CURLOPT_HTTPHEADER, _hlist)); - _curlEasyErrorChecked("curl_easy_setopt(CURLOPT_FAILONERROR)", curl_easy_setopt(_hcurl, CURLOPT_FAILONERROR, 1L)); _curlEasyErrorChecked("curl_easy_setopt(CURLOPT_WRITEFUNCTION)", diff --git a/src/http/Module.h b/src/http/Module.h index b97f4c47e3..6bd149fb16 100644 --- a/src/http/Module.h +++ b/src/http/Module.h @@ -92,9 +92,6 @@ class Module : public BaseModule { */ virtual void sendResponse(std::string const& content, std::string const& contentType) = 0; - std::string authKey() const { return _authKey; } // &&&uj - - private: /** * Pull the raw request body and translate it into a JSON object. diff --git a/src/http/RequestBodyJSON.h b/src/http/RequestBodyJSON.h index b88858a1dd..74c19a29d5 100644 --- a/src/http/RequestBodyJSON.h +++ b/src/http/RequestBodyJSON.h @@ -32,6 +32,8 @@ // This header declarations namespace lsst::qserv::http { +// TODO:UJ This should be renamed RequestBodyJson, coding standards. + /** * Class RequestBodyJSON represents the request body parsed into a JSON object. * This type of an object is only available for requests that have the following @@ -42,16 +44,16 @@ class RequestBodyJSON { /// parsed body of the request nlohmann::json objJson = nlohmann::json::object(); - RequestBody() = default; - RequestBody(RequestBody const&) = default; - RequestBody& operator=(RequestBody const&) = default; + RequestBodyJSON() = default; + RequestBodyJSON(RequestBodyJSON const&) = default; + RequestBodyJSON& operator=(RequestBodyJSON const&) = default; - ~RequestBody() = default; + ~RequestBodyJSON() = default; /// Make a new RequestBody based on `js` /// TODO:UJ This would be much more efficient if this class had objJson defined as /// a const reference or pointer to const, but implementation is likely ugly. - RequestBody(nlohmann::json const& js) : objJson(js) {} + RequestBodyJSON(nlohmann::json const& js) : objJson(js) {} /** * Check if the specified parameter is present in the input JSON object. @@ -84,15 +86,11 @@ class RequestBodyJSON { throw std::invalid_argument("RequestBodyJSON::" + std::string(__func__) + "[static] parameter 'obj' is not a valid JSON object"); } -<<<<<<< HEAD:src/http/RequestBodyJSON.h - if (obj.find(name) != obj.end()) return obj[name]; - throw std::invalid_argument("RequestBodyJSON::" + std::string(__func__) + -======= + if (auto const iter = obj.find(name); iter != obj.end()) { return *iter; } throw std::invalid_argument("RequestBody::" + std::string(__func__) + ->>>>>>> 4c670c16d (Czar and workers can send http messages to each other.):src/http/RequestBody.h "[static] required parameter " + name + " is missing in the request body"); } diff --git a/src/proto/CMakeLists.txt b/src/proto/CMakeLists.txt index 7f10de2e62..34fa9f77bb 100644 --- a/src/proto/CMakeLists.txt +++ b/src/proto/CMakeLists.txt @@ -11,25 +11,9 @@ target_sources(proto PRIVATE ${PROTO_PB_HDRS} FrameBuffer.cc ProtoHeaderWrap.cc - ScanTableInfo.cc ) target_link_libraries(proto PUBLIC log protobuf ) - -install( - TARGETS proto -) - -add_executable(testProtocol testProtocol.cc) - -target_link_libraries(testProtocol - proto - crypto - Boost::unit_test_framework -) - -add_test(NAME testProtocol COMMAND testProtocol) - diff --git a/src/proto/FakeProtocolFixture.h b/src/proto/FakeProtocolFixture.h deleted file mode 100644 index e4c232edaa..0000000000 --- a/src/proto/FakeProtocolFixture.h +++ /dev/null @@ -1,92 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_PROTO_FAKEPROTOCOLFIXTURE_H -#define LSST_QSERV_PROTO_FAKEPROTOCOLFIXTURE_H - -// System headers -#include -#include - -namespace lsst::qserv::proto { - -/// FakeProtocolFixture is a utility class containing code for making fake -/// versions of the protobufs messages used in Qserv. Its intent was -/// only to be used for test code. -class FakeProtocolFixture { -public: - FakeProtocolFixture() : _counter(0) {} - - TaskMsg* makeTaskMsg() { - TaskMsg* t(new TaskMsg()); - t->set_chunkid(20 + _counter); - t->set_db("elephant"); - t->set_jobid(0); - t->set_queryid(49); - t->set_scaninteractive(true); - - auto sTbl = t->add_scantable(); - sTbl->set_db("orange"); - sTbl->set_table("cart"); - sTbl->set_lockinmemory(false); - sTbl->set_scanrating(1); - - sTbl = t->add_scantable(); - sTbl->set_db("plum"); - sTbl->set_table("bike"); - sTbl->set_lockinmemory(false); - sTbl->set_scanrating(1); - - for (int i = 0; i < 3; ++i) { - TaskMsg::Fragment* f = t->add_fragment(); - f->add_query("Hello, this is a query."); - addSubChunk(*f, 100 + i); - f->set_resulttable("r_341"); - } - ++_counter; - return t; - } - - void addSubChunk(TaskMsg_Fragment& f, int scId) { - TaskMsg_Subchunk* s; - if (!f.has_subchunks()) { - TaskMsg_Subchunk subc; - // f.add_scgroup(); // How do I add optional objects? - subc.set_database("subdatabase_default"); - proto::TaskMsg_Subchunk_DbTbl* dbTbl = subc.add_dbtbl(); - dbTbl->set_db("subdatabase"); - dbTbl->set_tbl("subtable"); - f.mutable_subchunks()->CopyFrom(subc); - s = f.mutable_subchunks(); - } - s = f.mutable_subchunks(); - s->add_id(scId); - } - -private: - int _counter; -}; - -} // namespace lsst::qserv::proto - -#endif // #define LSST_QSERV_PROTO_FAKEPROTOCOLFIXTURE_H diff --git a/src/proto/ProtoImporter.h b/src/proto/ProtoImporter.h deleted file mode 100644 index 4173d7cfe9..0000000000 --- a/src/proto/ProtoImporter.h +++ /dev/null @@ -1,63 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2017 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_PROTO_PROTOIMPORTER_H -#define LSST_QSERV_PROTO_PROTOIMPORTER_H - -// System headers -#include -#include - -namespace lsst::qserv::proto { - -/// ProtoImporter -/// Minimal-copy import of an arbitrary proto msg from a raw buffer. -/// Example: -/// struct TaskMsgAcceptor : public ProtoImporter { -/// virtual void operator()(std::shared_ptr m) { ...} -/// }; -/// ProtoImporter p(std::shared_ptr()); -/// p(data,size); // calls operator() defined above. -template -class ProtoImporter { -public: - ProtoImporter() {} - - bool messageAcceptable(std::string const& msg) { - Msg m; - return setMsgFrom(m, msg.data(), msg.size()); - } - - static bool setMsgFrom(Msg& m, char const* buf, int bufLen) { - // For dev/debugging: accepts a partially-formed message - // bool ok = m.ParsePartialFromArray(buf, bufLen); - - // Accept only complete, compliant messages. - bool ok = m.ParseFromArray(buf, bufLen); - return ok && m.IsInitialized(); - } -}; - -} // namespace lsst::qserv::proto - -#endif // #define LSST_QSERV_PROTO_PROTOIMPORTER_H diff --git a/src/proto/testProtocol.cc b/src/proto/testProtocol.cc deleted file mode 100644 index 175eeeb98b..0000000000 --- a/src/proto/testProtocol.cc +++ /dev/null @@ -1,183 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2011-2016 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// System headers -#include -#include -#include -#include -#include - -// Third-party headers -#include -#include - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "proto/ScanTableInfo.h" -#include "proto/worker.pb.h" - -#include "proto/FakeProtocolFixture.h" - -// Boost unit test header -#define BOOST_TEST_MODULE Protocol_1 -#include - -namespace test = boost::test_tools; -namespace gio = google::protobuf::io; - -using namespace lsst::qserv; - -struct ProtocolFixture : public lsst::qserv::proto::FakeProtocolFixture { - ProtocolFixture(void) : FakeProtocolFixture(), counter(0) {} - ~ProtocolFixture(void) {} - - bool compareTaskMsgs(lsst::qserv::proto::TaskMsg& t1, lsst::qserv::proto::TaskMsg& t2) { - bool nonFragEq = (t1.chunkid() == t2.chunkid()) && (t1.db() == t2.db()); - bool sTablesEq = t1.scantable_size() == t2.scantable_size(); - for (int i = 0; i < t1.scantable_size(); ++i) { - auto const& sTbl1 = t1.scantable(i); - auto const& sTbl2 = t2.scantable(i); - bool eq = (sTbl1.db().compare(sTbl2.db()) == 0 && sTbl1.table() == sTbl2.table() && - sTbl1.lockinmemory() == sTbl2.lockinmemory() && - sTbl1.scanrating() == sTbl2.scanrating()); - sTablesEq = sTablesEq && eq; - } - - bool fEqual = (t1.fragment_size() == t2.fragment_size()); - for (int i = 0; i < t1.fragment_size(); ++i) { - fEqual = fEqual && compareFragment(t1.fragment(i), t2.fragment(i)); - } - return nonFragEq && fEqual && sTablesEq; - } - - bool compareSubchunk(lsst::qserv::proto::TaskMsg_Subchunk const& s1, - lsst::qserv::proto::TaskMsg_Subchunk const& s2) { - if (s1.database() != s2.database()) { - return false; - } - if (s1.dbtbl_size() != s2.dbtbl_size()) { - return false; - } - for (int i = 0; i < s1.dbtbl_size(); ++i) { - if (s1.dbtbl(i).db() != s2.dbtbl(i).db() && s1.dbtbl(i).tbl() != s2.dbtbl(i).tbl()) return false; - } - if (s1.id_size() != s2.id_size()) { - return false; - } - for (int i = 0; i < s1.id_size(); ++i) { - if (s1.id(i) != s2.id(i)) return false; - } - return true; - } - - bool compareFragment(lsst::qserv::proto::TaskMsg_Fragment const& f1, - lsst::qserv::proto::TaskMsg_Fragment const& f2) { - bool qEqual = true; - if (f1.query_size() == f2.query_size()) { - for (int i = 0; i < f1.query_size(); ++i) { - if (f1.query(i) != f2.query(i)) return false; - } - } else { - return false; - } - bool sEqual = true; - if (f1.has_subchunks()) { - if (f2.has_subchunks()) { - sEqual = sEqual && compareSubchunk(f1.subchunks(), f2.subchunks()); - } else { - sEqual = false; - } - } else if (f2.has_subchunks()) { - sEqual = false; - } - return qEqual && sEqual; - } - - int counter; -}; - -BOOST_FIXTURE_TEST_SUITE(ProtocolTestSuite, ProtocolFixture) - -BOOST_AUTO_TEST_CASE(TaskMsgMsgSanity) { - GOOGLE_PROTOBUF_VERIFY_VERSION; - std::stringstream ss; - std::unique_ptr t1(makeTaskMsg()); - BOOST_CHECK(t1.get()); - t1->SerializeToOstream(&ss); - - std::string blah = ss.str(); - std::stringstream ss2(blah); - std::unique_ptr t2(new lsst::qserv::proto::TaskMsg()); - BOOST_CHECK(t1.get()); - t2->ParseFromIstream(&ss2); - BOOST_CHECK(compareTaskMsgs(*t1, *t2)); -} - -BOOST_AUTO_TEST_CASE(ScanTableInfo) { - lsst::qserv::proto::ScanTableInfo stiA{"dba", "fruit", false, 1}; - lsst::qserv::proto::ScanTableInfo stiB{"dba", "fruit", true, 1}; - BOOST_CHECK(stiA.compare(stiB) < 0); - BOOST_CHECK(stiB.compare(stiA) > 0); - BOOST_CHECK(stiA.compare(stiA) == 0); - BOOST_CHECK(stiB.compare(stiB) == 0); - - lsst::qserv::proto::ScanTableInfo stiC{"dba", "fruit", true, 1}; - lsst::qserv::proto::ScanTableInfo stiD{"dba", "fruit", true, 2}; - BOOST_CHECK(stiC.compare(stiD) < 0); - BOOST_CHECK(stiD.compare(stiC) > 0); - BOOST_CHECK(stiC.compare(stiC) == 0); - BOOST_CHECK(stiD.compare(stiD) == 0); - - lsst::qserv::proto::ScanTableInfo stiE{"dba", "fruit", true, 2}; - lsst::qserv::proto::ScanTableInfo stiF{"dbb", "fruit", true, 2}; - BOOST_CHECK(stiE.compare(stiF) < 0); - BOOST_CHECK(stiF.compare(stiE) > 0); - BOOST_CHECK(stiE.compare(stiE) == 0); - BOOST_CHECK(stiF.compare(stiF) == 0); - - lsst::qserv::proto::ScanTableInfo stiG{"dbb", "fruit", true, 2}; - lsst::qserv::proto::ScanTableInfo stiH{"dbb", "veggie", true, 2}; - BOOST_CHECK(stiG.compare(stiH) < 0); - BOOST_CHECK(stiH.compare(stiG) > 0); - BOOST_CHECK(stiG.compare(stiG) == 0); - BOOST_CHECK(stiH.compare(stiH) == 0); - - lsst::qserv::proto::ScanTableInfo::ListOf list = {stiE, stiH, stiC, stiD, stiB, stiA, stiG, stiF}; - lsst::qserv::proto::ScanInfo scanInfo; - scanInfo.infoTables = list; - scanInfo.sortTablesSlowestFirst(); - int j = 0; - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiH) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiG) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiF) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiE) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiD) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiC) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiB) == 0); - BOOST_CHECK(scanInfo.infoTables[j++].compare(stiA) == 0); -} - -BOOST_AUTO_TEST_SUITE_END() diff --git a/src/proto/worker.proto b/src/proto/worker.proto index 46f3e69efd..1ccefaf5f9 100644 --- a/src/proto/worker.proto +++ b/src/proto/worker.proto @@ -29,53 +29,6 @@ option cc_enable_arenas = true; package lsst.qserv.proto; -// TODO:UJ delete when xrootd removed. ResonseSummary will need to be kept. -// Query message sent to worker -// One of these Task objects should be sent. -message TaskMsg { - // Future: might have multiple db/chunk dependencies. - optional string db = 2; - optional int32 chunkid = 3; - // repeated string scantables = 4; // obsolete - optional string user = 6; - optional int32 scanpriority = 8; - message Subchunk { - optional string database = 1; // database (unused) - repeated DbTbl dbtbl = 2; // subchunked tables - repeated int32 id = 3; // subchunk ids - message DbTbl { - required string db = 1; - required string tbl = 2; - } - } - message Fragment { - // A query fragment without "CREATE or INSERT". - // Worker should synthesize. - repeated string query = 1; - optional string resulttable = 3; - optional Subchunk subchunks = 4; // Only needed with subchunk-ed queries - - // Each fragment may only write results to one table, - // but multiple fragments may write to the same table, - // in which case the table contains a concatenation of the - // contributing fragments' rows. - } - repeated Fragment fragment = 5; - message ScanTable { - required string db = 1; - required string table = 2; - required bool lockInMemory = 3; - required int32 scanRating = 4; - } - repeated ScanTable scantable = 9; - optional uint64 queryid = 10; - optional int32 jobid = 11; - optional bool scaninteractive = 12; - optional int32 attemptcount = 13; - optional uint32 czarid = 14; - optional int32 maxtablesize_mb = 15 [default = 0]; -} - // The file-based result delivery protocol has two kinds of messages. // // 1. The summary message sent back to Czar over the XROOTD/SSI protocol: @@ -113,34 +66,3 @@ message ResponseData { required uint32 rowcount = 2; required uint64 transmitsize = 3; } - -///////////////////////////////////////////////////////////////// -// Protocol definition for the query management requests. These -// requests do not require any response messages to be explicitly -// sent by workers. -// -// ATTENTION: each message sent to a worker must be preceeded by -// an int32 size (network-byte-ordered) word carrying a size -// of the message. -//////////////////////////////////////////////////////////////// - -// The completion status to be sent back with responses to the query management requests. -message WorkerCommandStatus { - enum Code { - SUCCESS = 1; // The successful completion of a request. - ERROR = 2; // An error occurred during request execution. - } - optional Code code = 3 [default = SUCCESS]; - optional string error = 2 [default = ""]; // Optional error message (depends on the code) -} - -message QueryManagement { - enum Operation { - CANCEL_AFTER_RESTART = 1; // Cancel older queries before the specified query (excluding that one). - CANCEL = 2; // Cancel a specific query. - COMPLETE = 3; // Notify workers on the completion of the specified query. - } - required Operation op = 1; - required uint64 czar_id = 3; - required uint64 query_id = 2; -} diff --git a/src/protojson/CMakeLists.txt b/src/protojson/CMakeLists.txt new file mode 100644 index 0000000000..8ac88b4cda --- /dev/null +++ b/src/protojson/CMakeLists.txt @@ -0,0 +1,40 @@ +add_library(protojson SHARED) + +target_sources(protojson PRIVATE + ScanTableInfo.cc + UberJobMsg.cc + WorkerQueryStatusData.cc +) + +target_link_libraries(protojson PUBLIC + curl + http + log + qhttp + util + Boost::filesystem + Boost::regex + Boost::system + cpp-httplib +) + +install(TARGETS protojson) + +function(PROTOJSON_TESTS) + foreach(TEST IN ITEMS ${ARGV}) + add_executable(${TEST} ${TEST}.cc) + target_link_libraries(${TEST} PUBLIC + global + http + protojson + Boost::unit_test_framework + Threads::Threads + ) + add_test(NAME ${TEST} COMMAND ${TEST}) + endforeach() +endfunction() + +protojson_tests( + testStatusData + testUberJobMsg +) diff --git a/src/proto/ScanTableInfo.cc b/src/protojson/ScanTableInfo.cc similarity index 69% rename from src/proto/ScanTableInfo.cc rename to src/protojson/ScanTableInfo.cc index 101e1a8d77..32da583bf2 100644 --- a/src/proto/ScanTableInfo.cc +++ b/src/protojson/ScanTableInfo.cc @@ -22,16 +22,27 @@ */ // Class header -#include "proto/ScanTableInfo.h" +#include "protojson/ScanTableInfo.h" // System headers #include #include // Qserv headers +#include "http/RequestBodyJSON.h" #include "util/IterableFormatter.h" -namespace lsst::qserv::proto { +// LSST headers +#include "lsst/log/Log.h" + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.ScanTableInfo"); +} // namespace + +using namespace std; +using namespace nlohmann; + +namespace lsst::qserv::protojson { /// @return 0 if equal, -1 if this < rhs, 1 if this > rhs int ScanTableInfo::compare(ScanTableInfo const& rhs) const { @@ -87,6 +98,39 @@ void ScanInfo::sortTablesSlowestFirst() { std::sort(infoTables.begin(), infoTables.end(), func); } +nlohmann::json ScanInfo::serializeJson() const { + auto jsScanInfo = json({{"infoscanrating", scanRating}, {"infotables", json::array()}}); + + auto& jsInfoTables = jsScanInfo["infotables"]; + for (auto const& tInfo : infoTables) { + json jsTInfo = json({{"sidb", tInfo.db}, + {"sitable", tInfo.table}, + {"sirating", tInfo.scanRating}, + {"silockinmem", tInfo.lockInMemory}}); + + jsInfoTables.push_back(jsTInfo); + } + + return jsScanInfo; +} + +ScanInfo::Ptr ScanInfo::createFromJson(nlohmann::json const& siJson) { + Ptr siPtr = create(); + auto& iTbls = siPtr->infoTables; + + siPtr->scanRating = http::RequestBodyJSON::required(siJson, "infoscanrating"); + json const& jsTbls = http::RequestBodyJSON::required(siJson, "infotables"); + for (auto const& jsElem : jsTbls) { + auto db = http::RequestBodyJSON::required(jsElem, "sidb"); + auto table = http::RequestBodyJSON::required(jsElem, "sitable"); + auto sRating = http::RequestBodyJSON::required(jsElem, "sirating"); + auto lockInMem = http::RequestBodyJSON::required(jsElem, "silockinmem"); + iTbls.emplace_back(db, table, lockInMem, sRating); + } + + return siPtr; +} + std::ostream& operator<<(std::ostream& os, ScanTableInfo const& tbl) { os << "(db=" << tbl.db << " table=" << tbl.table; os << " lockInMemory=" << tbl.lockInMemory << " scanRating=" << tbl.scanRating << ")"; @@ -98,4 +142,4 @@ std::ostream& operator<<(std::ostream& os, ScanInfo const& info) { return os; } -} // namespace lsst::qserv::proto +} // namespace lsst::qserv::protojson diff --git a/src/proto/ScanTableInfo.h b/src/protojson/ScanTableInfo.h similarity index 73% rename from src/proto/ScanTableInfo.h rename to src/protojson/ScanTableInfo.h index f2dacec61a..061ea0c0f9 100644 --- a/src/proto/ScanTableInfo.h +++ b/src/protojson/ScanTableInfo.h @@ -21,17 +21,18 @@ * see . */ -#ifndef LSST_QSERV_PROTO_SCANTABLEINFO_H -#define LSST_QSERV_PROTO_SCANTABLEINFO_H +#ifndef LSST_QSERV_PROTOJSON_SCANTABLEINFO_H +#define LSST_QSERV_PROTOJSON_SCANTABLEINFO_H // System headers +#include #include #include -// Qserv headers -#include "proto/worker.pb.h" +// Third party headers +#include "nlohmann/json.hpp" -namespace lsst::qserv::proto { +namespace lsst::qserv::protojson { /// Structure to store shared scan information for a single table. /// @@ -42,22 +43,9 @@ struct ScanTableInfo { ScanTableInfo(std::string const& db_, std::string const& table_) : db(db_), table(table_) {} ScanTableInfo(std::string const& db_, std::string const& table_, bool lockInMemory_, int scanRating_) : db{db_}, table{table_}, lockInMemory{lockInMemory_}, scanRating{scanRating_} {} - ScanTableInfo(TaskMsg_ScanTable const& scanTbl) - : db{scanTbl.db()}, - table{scanTbl.table()}, - lockInMemory{scanTbl.lockinmemory()}, - scanRating{scanTbl.scanrating()} {} ScanTableInfo(ScanTableInfo const&) = default; - /// Copy contents of this object into a TaskMsg_ScanTable object. - void copyToScanTable(TaskMsg_ScanTable* msgScanTbl) const { - msgScanTbl->set_db(db); - msgScanTbl->set_table(table); - msgScanTbl->set_lockinmemory(lockInMemory); - msgScanTbl->set_scanrating(scanRating); - } - int compare(ScanTableInfo const& rhs) const; std::string db; @@ -66,13 +54,25 @@ struct ScanTableInfo { int scanRating{0}; }; -struct ScanInfo { +/// This class stores information about database table ratings for +/// a user query. +class ScanInfo { +public: + using Ptr = std::shared_ptr; + /// Threshold priority values. Scan priorities are not limited to these values. enum Rating { FASTEST = 0, FAST = 10, MEDIUM = 20, SLOW = 30, SLOWEST = 100 }; ScanInfo() = default; ScanInfo(ScanInfo const&) = default; + static Ptr create() { return Ptr(new ScanInfo()); } + + static Ptr createFromJson(nlohmann::json const& ujJson); + + /// Return a json version of the contents of this class. + nlohmann::json serializeJson() const; + void sortTablesSlowestFirst(); int compareTables(ScanInfo const& rhs); @@ -83,6 +83,6 @@ struct ScanInfo { std::ostream& operator<<(std::ostream& os, ScanTableInfo const& tbl); std::ostream& operator<<(std::ostream& os, ScanInfo const& info); -} // namespace lsst::qserv::proto +} // namespace lsst::qserv::protojson -#endif // LSST_QSERV_PROTO_SCANTABLEINFO_H +#endif // LSST_QSERV_PROTOJSON_SCANTABLEINFO_H diff --git a/src/protojson/UberJobMsg.cc b/src/protojson/UberJobMsg.cc new file mode 100644 index 0000000000..65564cdf48 --- /dev/null +++ b/src/protojson/UberJobMsg.cc @@ -0,0 +1,511 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "protojson/UberJobMsg.h" + +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "qdisp/JobQuery.h" +#include "qdisp/JobDescription.h" +#include "qproc/ChunkQuerySpec.h" +#include "util/common.h" +#include "util/TimeUtils.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.UberJobMsg"); +} // namespace + +namespace lsst::qserv::protojson { + +UberJobMsg::UberJobMsg(unsigned int metaVersion, std::string const& replicationInstanceId, + std::string const& replicationAuthKey, CzarContactInfo::Ptr const& czInfo, + string const& workerId, QueryId qId, UberJobId ujId, int rowLimit, int maxTableSizeMB, + ScanInfo::Ptr const& scanInfo_, + std::vector> const& jobs) + : _metaVersion(metaVersion), + _replicationInstanceId(replicationInstanceId), + _replicationAuthKey(replicationAuthKey), + _czInfo(czInfo), + _workerId(workerId), + _qId(qId), + _ujId(ujId), + _rowLimit(rowLimit), + _maxTableSizeMB(maxTableSizeMB), + _scanInfo(scanInfo_) { + + for (auto& jobPtr : jobs) { + // This creates the JobMsg objects for all relates jobs and their fragments. + auto jobMsg = JobMsg::create(jobPtr, _jobSubQueryTempMap, _jobDbTablesMap); + _jobMsgVect->push_back(jobMsg); + } +} + +json UberJobMsg::serializeJson() const { + json ujmJson = {{"version", _metaVersion}, + {"instance_id", _replicationInstanceId}, + {"auth_key", _replicationAuthKey}, + {"worker", _workerId}, + {"queryid", _qId}, + {"uberjobid", _ujId}, + {"czarinfo", _czInfo->serializeJson()}, + {"rowlimit", _rowLimit}, + {"subqueries_map", _jobSubQueryTempMap->serializeJson()}, + {"dbtables_map", _jobDbTablesMap->serializeJson()}, + {"maxtablesizemb", _maxTableSizeMB}, + {"scaninfo", _scanInfo->serializeJson()}, + {"jobs", json::array()}}; + + auto& jsJobs = ujmJson["jobs"]; + for (auto const& jbMsg : *_jobMsgVect) { + jsJobs.emplace_back(jbMsg->serializeJson()); + } + + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " &&& ujmJson=" << ujmJson); + return ujmJson; +} + +UberJobMsg::Ptr UberJobMsg::createFromJson(nlohmann::json const& ujmJson) { + LOGS(_log, LOG_LVL_TRACE, "UberJobMsg::createFromJson ujmJson=" << ujmJson); + try { + if (ujmJson["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, "UberJobMsg::createFromJson bad version " << ujmJson["version"]); + return nullptr; + } + + auto czInfo_ = CzarContactInfo::createFromJson(ujmJson["czarinfo"]); + if (czInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "UberJobMsg::createFromJson czar could not be parsed in " << ujmJson); + return nullptr; + } + + auto scanInfo_ = ScanInfo::createFromJson(ujmJson["scaninfo"]); + if (scanInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, + "UberJobMsg::createFromJson scanInfo could not be parsed in " << ujmJson); + return nullptr; + } + + auto metaVersion = http::RequestBodyJSON::required(ujmJson, "version"); + auto replicationInstanceId = http::RequestBodyJSON::required(ujmJson, "instance_id"); + auto replicationAuthKey = http::RequestBodyJSON::required(ujmJson, "auth_key"); + auto workerId = http::RequestBodyJSON::required(ujmJson, "worker"); + auto qId = http::RequestBodyJSON::required(ujmJson, "queryid"); + auto ujId = http::RequestBodyJSON::required(ujmJson, "uberjobid"); + auto rowLimit = http::RequestBodyJSON::required(ujmJson, "rowlimit"); + auto maxTableSizeMB = http::RequestBodyJSON::required(ujmJson, "maxtablesizemb"); + auto czInfo = CzarContactInfo::createFromJson(ujmJson["czarinfo"]); + auto jsUjJobs = http::RequestBodyJSON::required(ujmJson, "jobs"); + + std::vector> emptyJobs; + + Ptr ujmPtr = Ptr(new UberJobMsg(metaVersion, replicationInstanceId, replicationAuthKey, czInfo, + workerId, qId, ujId, rowLimit, maxTableSizeMB, scanInfo_, emptyJobs)); + + auto const& jsSubQueriesMap = http::RequestBodyJSON::required(ujmJson, "subqueries_map"); + ujmPtr->_jobSubQueryTempMap = JobSubQueryTempMap::createFromJson(jsSubQueriesMap); + + auto jsDbTablesMap = http::RequestBodyJSON::required(ujmJson, "dbtables_map"); + ujmPtr->_jobDbTablesMap = JobDbTablesMap::createFromJson(jsDbTablesMap); + + for (auto const& jsUjJob : jsUjJobs) { + JobMsg::Ptr jobMsgPtr = + JobMsg::createFromJson(jsUjJob, ujmPtr->_jobSubQueryTempMap, ujmPtr->_jobDbTablesMap); + ujmPtr->_jobMsgVect->push_back(jobMsgPtr); + } + return ujmPtr; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, "UberJobMsg::createFromJson invalid " << exc.what() << " json=" << ujmJson); + } + return nullptr; +} + +JobMsg::Ptr JobMsg::create(std::shared_ptr const& jobPtr, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& jobDbTablesMap) { + auto jMsg = Ptr(new JobMsg(jobPtr, jobSubQueryTempMap, jobDbTablesMap)); + return jMsg; +} + +JobMsg::JobMsg(std::shared_ptr const& jobPtr, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap) + : _jobSubQueryTempMap(jobSubQueryTempMap), _jobDbTablesMap(jobDbTablesMap) { + auto const descr = jobPtr->getDescription(); + if (descr == nullptr) { + throw util::Bug(ERR_LOC, cName(__func__) + " description=null for job=" + jobPtr->getIdStr()); + } + auto chunkQuerySpec = descr->getChunkQuerySpec(); + _jobId = descr->id(); + _attemptCount = descr->getAttemptCount(); + _chunkQuerySpecDb = chunkQuerySpec->db; + _scanRating = chunkQuerySpec->scanInfo->scanRating; + _scanInteractive = chunkQuerySpec->scanInteractive; + _chunkId = chunkQuerySpec->chunkId; + + // Add scan tables (TODO:UJ Verify this is the same for all jobs.) + for (auto const& sTbl : chunkQuerySpec->scanInfo->infoTables) { + int index = jobDbTablesMap->findDbTable(make_pair(sTbl.db, sTbl.table)); + jobDbTablesMap->setScanRating(index, sTbl.scanRating, sTbl.lockInMemory); + _chunkScanTableIndexes.push_back(index); + } + + // Add fragments + _jobFragments = JobFragment::createVect(*chunkQuerySpec, jobSubQueryTempMap, jobDbTablesMap); +} + +nlohmann::json JobMsg::serializeJson() const { + auto jsJobMsg = nlohmann::json({{"jobId", _jobId}, + {"attemptCount", _attemptCount}, + {"querySpecDb", _chunkQuerySpecDb}, + {"scanPriority", _scanRating}, + {"scanInteractive", _scanInteractive}, + {"chunkId", _chunkId}, + {"chunkscantables_indexes", nlohmann::json::array()}, + {"queryFragments", json::array()}}); + + // These are indexes into _jobDbTablesMap, which is shared between all JobMsg in this UberJobMsg. + // &&& TODO:UJ queries appear to work even when "chunkscantables_indexes" is wrong + auto& jsqCstIndexes = jsJobMsg["chunkscantables_indexes"]; + for (auto const& index : _chunkScanTableIndexes) { + jsqCstIndexes.push_back(index); + } + + auto& jsqFrags = jsJobMsg["queryFragments"]; + for (auto& jFrag : *_jobFragments) { + jsqFrags.emplace_back(jFrag->serializeJson()); + } + + return jsJobMsg; +} + +JobMsg::JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap, + JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int scanRating, + bool scanInteractive, int chunkId) + : _jobId(jobId), + _attemptCount(attemptCount), + _chunkQuerySpecDb(chunkQuerySpecDb), + _scanRating(scanRating), + _scanInteractive(scanInteractive), + _chunkId(chunkId), + _jobSubQueryTempMap(jobSubQueryTempMap), + _jobDbTablesMap(jobDbTablesMap) {} + +JobMsg::Ptr JobMsg::createFromJson(nlohmann::json const& ujJson, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& jobDbTablesMap) { + JobId jobId = http::RequestBodyJSON::required(ujJson, "jobId"); + int attemptCount = http::RequestBodyJSON::required(ujJson, "attemptCount"); + string chunkQuerySpecDb = http::RequestBodyJSON::required(ujJson, "querySpecDb"); + int scanRating = http::RequestBodyJSON::required(ujJson, "scanPriority"); + bool scanInteractive = http::RequestBodyJSON::required(ujJson, "scanInteractive"); + int chunkId = http::RequestBodyJSON::required(ujJson, "chunkId"); + + json jsQFrags = http::RequestBodyJSON::required(ujJson, "queryFragments"); + + Ptr jMsgPtr = Ptr(new JobMsg(jobSubQueryTempMap, jobDbTablesMap, jobId, attemptCount, chunkQuerySpecDb, + scanRating, scanInteractive, chunkId)); + json jsChunkTblIndexes = http::RequestBodyJSON::required(ujJson, "chunkscantables_indexes"); + jMsgPtr->_chunkScanTableIndexes = jsChunkTblIndexes.get>(); + jMsgPtr->_jobFragments = + JobFragment::createVectFromJson(jsQFrags, jMsgPtr->_jobSubQueryTempMap, jMsgPtr->_jobDbTablesMap); + + return jMsgPtr; +} + +json JobSubQueryTempMap::serializeJson() const { + // std::map _qTemplateMap; + json jsSubQueryTemplateMap = {{"subquerytemplate_map", json::array()}}; + auto& jsSqtMap = jsSubQueryTemplateMap["subquerytemplate_map"]; + for (auto const& [key, templ] : _qTemplateMap) { + json jsElem = {{"index", key}, {"template", templ}}; + jsSqtMap.push_back(jsElem); + } + + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " " << jsSqtMap); + return jsSubQueryTemplateMap; +} + +JobSubQueryTempMap::Ptr JobSubQueryTempMap::createFromJson(nlohmann::json const& ujJson) { + Ptr sqtMapPtr = create(); + auto& sqtMap = sqtMapPtr->_qTemplateMap; + LOGS(_log, LOG_LVL_TRACE, "JobSubQueryTempMap::createFromJson " << ujJson); + auto const& jsElements = ujJson["subquerytemplate_map"]; + for (auto const& jsElem : jsElements) { + int index = http::RequestBodyJSON::required(jsElem, "index"); + string templ = http::RequestBodyJSON::required(jsElem, "template"); + auto res = sqtMap.insert(make_pair(index, templ)); + if (!res.second) { + throw invalid_argument(sqtMapPtr->cName(__func__) + "index=" + to_string(index) + "=" + templ + + " index already found in " + to_string(ujJson)); + } + } + return sqtMapPtr; +} + +int JobSubQueryTempMap::findSubQueryTemp(string const& qTemp) { + // The expected number of templates is expected to be small, less than 4, + // so this shouldn't be horribly expensive. + for (auto const& [key, temp] : _qTemplateMap) { + if (temp == qTemp) { + return key; + } + } + + // Need to insert + int index = _qTemplateMap.size(); + _qTemplateMap[index] = qTemp; + return index; +} + +int JobDbTablesMap::findDbTable(pair const& dbTablePair) { + // The expected number of templates is expected to be small, less than 4, + // so this shouldn't be horribly expensive. + for (auto const& [key, dbTbl] : _dbTableMap) { + if (dbTablePair == dbTbl) { + return key; + } + } + + // Need to insert + int index = _dbTableMap.size(); + _dbTableMap[index] = dbTablePair; + return index; +} + +json JobDbTablesMap::serializeJson() const { + json jsDbTablesMap = {{"dbtable_map", json::array()}, {"scanrating_map", json::array()}}; + + auto& jsDbTblMap = jsDbTablesMap["dbtable_map"]; + for (auto const& [key, valPair] : _dbTableMap) { + json jsDbTbl = {{"index", key}, {"db", valPair.first}, {"table", valPair.second}}; + jsDbTblMap.push_back(jsDbTbl); + } + + auto& jsScanRatingMap = jsDbTablesMap["scanrating_map"]; + for (auto const& [key, valPair] : _scanRatingMap) { + json jsScanR = {{"index", key}, {"scanrating", valPair.first}, {"lockinmem", valPair.second}}; + jsScanRatingMap.push_back(jsScanR); + } + + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " " << jsDbTablesMap); + return jsDbTablesMap; +} + +JobDbTablesMap::Ptr JobDbTablesMap::createFromJson(nlohmann::json const& ujJson) { + Ptr dbTablesMapPtr = create(); + auto& dbTblMap = dbTablesMapPtr->_dbTableMap; + auto& scanRMap = dbTablesMapPtr->_scanRatingMap; + + LOGS(_log, LOG_LVL_TRACE, "JobDbTablesMap::createFromJson " << ujJson); + + json const& jsDbTbl = ujJson["dbtable_map"]; + for (auto const& jsElem : jsDbTbl) { + int index = http::RequestBodyJSON::required(jsElem, "index"); + string db = http::RequestBodyJSON::required(jsElem, "db"); + string tbl = http::RequestBodyJSON::required(jsElem, "table"); + auto res = dbTblMap.insert(make_pair(index, make_pair(db, tbl))); + if (!res.second) { + throw invalid_argument(dbTablesMapPtr->cName(__func__) + " index=" + to_string(index) + "=" + db + + +"." + tbl + " index already found in " + to_string(jsDbTbl)); + } + } + + json const& jsScanR = ujJson["scanrating_map"]; + for (auto const& jsElem : jsScanR) { + int index = http::RequestBodyJSON::required(jsElem, "index"); + int scanR = http::RequestBodyJSON::required(jsElem, "scanrating"); + bool lockInMem = http::RequestBodyJSON::required(jsElem, "lockinmem"); + auto res = scanRMap.insert(make_pair(index, make_pair(scanR, lockInMem))); + if (!res.second) { + throw invalid_argument(dbTablesMapPtr->cName(__func__) + " index=" + to_string(index) + "=" + + to_string(scanR) + +", " + to_string(lockInMem) + + " index already found in " + to_string(jsDbTbl)); + } + } + + return dbTablesMapPtr; +} + +void JobDbTablesMap::setScanRating(int index, int scanRating, bool lockInMemory) { + auto iter = _scanRatingMap.find(index); + if (iter == _scanRatingMap.end()) { + _scanRatingMap[index] = make_pair(scanRating, lockInMemory); + } else { + auto& elem = *iter; + auto& pr = elem.second; + auto& [sRating, lInMem] = pr; + if (sRating != scanRating || lInMem != lockInMemory) { + auto [dbName, tblName] = getDbTable(index); + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " unexpected change in scanRating for " << dbName << "." << tblName + << " from " << sRating << " to " << scanRating << " lockInMemory from " + << lInMem << " to " << lockInMemory); + if (scanRating > sRating) { + sRating = scanRating; + lInMem = lockInMemory; + } + } + } +} + +JobFragment::JobFragment(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& jobDbTablesMap) + : _jobSubQueryTempMap(jobSubQueryTempMap), _jobDbTablesMap(jobDbTablesMap) {} + +JobFragment::VectPtr JobFragment::createVect(qproc::ChunkQuerySpec const& chunkQuerySpec, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& jobDbTablesMap) { + VectPtr jFragments{new Vect()}; + if (chunkQuerySpec.nextFragment.get()) { + qproc::ChunkQuerySpec const* sPtr = &chunkQuerySpec; + while (sPtr) { + LOGS(_log, LOG_LVL_TRACE, "nextFragment"); + // Linked fragments will not have valid subChunkTables vectors, + // So, we reuse the root fragment's vector. + _addFragment(*jFragments, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, sPtr->queries, + jobSubQueryTempMap, jobDbTablesMap); + sPtr = sPtr->nextFragment.get(); + } + } else { + LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); + _addFragment(*jFragments, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, + chunkQuerySpec.queries, jobSubQueryTempMap, jobDbTablesMap); + } + + return jFragments; +} + +void JobFragment::_addFragment(std::vector& jFragments, DbTableSet const& subChunkTables, + std::vector const& subchunkIds, std::vector const& queries, + JobSubQueryTempMap::Ptr const& subQueryTemplates, + JobDbTablesMap::Ptr const& dbTablesMap) { + LOGS(_log, LOG_LVL_TRACE, "JobFragment::_addFragment start"); + Ptr jFrag = Ptr(new JobFragment(subQueryTemplates, dbTablesMap)); + + // queries: The query string is stored in `_jobSubQueryTempMap` and the list of + // integer indexes, `_subQueryTempIndexes`, points back to the specific template. + for (auto& qry : queries) { + int index = jFrag->_jobSubQueryTempMap->findSubQueryTemp(qry); + jFrag->_jobSubQueryTempIndexes.push_back(index); + LOGS(_log, LOG_LVL_TRACE, jFrag->cName(__func__) << " added frag=" << qry << " index=" << index); + } + + // Add the db+table pairs to the subchunks for the fragment. + for (auto& tbl : subChunkTables) { + int index = jFrag->_jobDbTablesMap->findDbTable(make_pair(tbl.db, tbl.table)); + jFrag->_jobDbTablesIndexes.push_back(index); + LOGS(_log, LOG_LVL_TRACE, + jFrag->cName(__func__) << " added dbtbl=" << tbl.db << "." << tbl.table << " index=" << index); + } + + // Add subchunk id numbers + for (auto& subchunkId : subchunkIds) { + jFrag->_subchunkIds.push_back(subchunkId); + LOGS(_log, LOG_LVL_TRACE, jFrag->cName(__func__) << " added subchunkId=" << subchunkId); + } + + jFragments.push_back(move(jFrag)); +} + +string JobFragment::dump() const { + stringstream os; + os << " templateIndexes={"; + for (int j : _jobSubQueryTempIndexes) { + os << j << ", "; + } + os << "} subchunkIds={"; + for (int j : _subchunkIds) { + os << j << ", "; + } + os << "} dbtbl={"; + for (int j : _subchunkIds) { + os << j << ", "; + } + os << "}"; + return os.str(); +} + +nlohmann::json JobFragment::serializeJson() const { + json jsFragment = {{"subquerytemplate_indexes", _jobSubQueryTempIndexes}, + {"dbtables_indexes", _jobDbTablesIndexes}, + {"subchunkids", _subchunkIds}}; + + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " " << jsFragment); + return jsFragment; +} + +JobFragment::VectPtr JobFragment::createVectFromJson(nlohmann::json const& jsFrags, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& dbTablesMap) { + LOGS(_log, LOG_LVL_TRACE, "JobFragment::createVectFromJson " << jsFrags); + + JobFragment::VectPtr jobFragments{new JobFragment::Vect()}; + + for (auto const& jsFrag : jsFrags) { + Ptr jobFrag = Ptr(new JobFragment(jobSubQueryTempMap, dbTablesMap)); + + jobFrag->_jobSubQueryTempIndexes = jsFrag["subquerytemplate_indexes"].get>(); + for (int j : jobFrag->_jobSubQueryTempIndexes) { + try { + string tem = jobSubQueryTempMap->getSubQueryTemp(j); + LOGS(_log, LOG_LVL_TRACE, jobFrag->cName(__func__) << " j=" << j << " =" << tem); + } catch (std::out_of_range const& ex) { + LOGS(_log, LOG_LVL_ERROR, + jobFrag->cName(__func__) << " index=" << j << " not found in template map " << jsFrag); + // rethrow as something callers expect. + throw std::invalid_argument(jobFrag->cName(__func__) + " template index=" + to_string(j) + + " " + ex.what()); + } + } + + jobFrag->_jobDbTablesIndexes = jsFrag["dbtables_indexes"].get>(); + for (int j : jobFrag->_jobDbTablesIndexes) { + try { + auto dbTblPr = dbTablesMap->getDbTable(j); + LOGS(_log, LOG_LVL_TRACE, + jobFrag->cName(__func__) + << " j=" << j << " =" << dbTblPr.first << "." << dbTblPr.second); + } catch (std::out_of_range const& ex) { + LOGS(_log, LOG_LVL_ERROR, + jobFrag->cName(__func__) << " index=" << j << " not found in dbTable map " << jsFrag); + // rethrow as something callers expect. + throw std::invalid_argument(jobFrag->cName(__func__) + " dbtable index=" + to_string(j) + + " " + ex.what()); + } + } + + jobFrag->_subchunkIds = jsFrag["subchunkids"].get>(); + jobFragments->push_back(jobFrag); + } + return jobFragments; +} + +} // namespace lsst::qserv::protojson diff --git a/src/protojson/UberJobMsg.h b/src/protojson/UberJobMsg.h new file mode 100644 index 0000000000..d5f6ade9e0 --- /dev/null +++ b/src/protojson/UberJobMsg.h @@ -0,0 +1,311 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_PROTOJSON_UBERJOBMSG_H +#define LSST_QSERV_PROTOJSON_UBERJOBMSG_H + +// System headers +#include +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "global/clock_defs.h" +#include "global/DbTable.h" +#include "global/intTypes.h" +#include "protojson/ScanTableInfo.h" +#include "protojson/WorkerQueryStatusData.h" + +namespace lsst::qserv::qdisp { +class JobQuery; +} + +namespace lsst::qserv::qproc { +class ChunkQuerySpec; +} + +// This header declarations +namespace lsst::qserv::protojson { + +/// This class is used to store query template strings names in a reasonably +/// concise fashion. +/// The same templates recur frequently, so the individual occurrences +/// will be replaced with an integer index and use this class to recover the +/// original template. +class JobSubQueryTempMap { +public: + using Ptr = std::shared_ptr; + + std::string cName(const char* fName) const { return std::string("JobSubQueryTempMap::") + fName; } + + JobSubQueryTempMap(JobSubQueryTempMap const&) = delete; + + static Ptr create() { return Ptr(new JobSubQueryTempMap()); } + + /// &&& doc + static Ptr createFromJson(nlohmann::json const& ujJson); + + /// Find or insert qTemp into the map and return its index. + int findSubQueryTemp(std::string const& qTemp); + + /// Return the SubQueryTemp string at `index`. + /// @throws std::out_of_range + std::string getSubQueryTemp(int index) { return _qTemplateMap.at(index); } + + nlohmann::json serializeJson() const; + +private: + JobSubQueryTempMap() = default; + + std::map _qTemplateMap; +}; + +/// This class is used to store db.table names in a reasonably concise fashion. +/// The same db+table name pairs recur frequently, so the individual occurrences +/// will be replaced with an integer index and use this class to recover the +/// complete names. +class JobDbTablesMap { // &&& this class can probably be deleted +public: + using Ptr = std::shared_ptr; + + std::string cName(const char* fName) const { return std::string("JobDbTablesMap::") + fName; } + + JobDbTablesMap(JobDbTablesMap const&) = delete; + + static Ptr create() { return Ptr(new JobDbTablesMap()); } + + /// &&& doc + static Ptr createFromJson(nlohmann::json const& ujJson); + + /// Find or insert the db.table pair into the map and return its index. + int findDbTable(std::pair const& dbTablePair); + + /// Return the db.table pair at `index`. + /// @throws std::out_of_range + std::pair getDbTable(int index) { return _dbTableMap.at(index); } + + /// &&& TODO:UJ compare with scan rating for entire UberJob + void setScanRating(int index, int scanRating, bool lockInMemory); + + /// Return scanRating(int) and lockInMemory(bool) for the dbTable at `index`. + /// TODO:UJ &&& lockInMemory is expected to go away. + std::pair getScanRating(int index) { return _scanRatingMap[index]; } + + nlohmann::json serializeJson() const; + +private: + JobDbTablesMap() = default; + + /// Map of db name and table name pairs: db first, table second. + /// The order in the map is arbitrary, but must be consistent + /// so that lookups using the int index always return the same pair. + std::map> _dbTableMap; + + /// Key is dbTable index, val is scanRating(int) lockInMemory(bool) + std::map> _scanRatingMap; +}; + +/// This class stores the contents of a query fragment, which will be reconstructed +/// and run on a worker to help answer a user query. +class JobFragment { +public: + using Ptr = std::shared_ptr; + using Vect = std::vector; + using VectPtr = std::shared_ptr; + + std::string cName(const char* fName) const { return std::string("JobFragment::") + fName; } + + JobFragment() = delete; + JobFragment(JobFragment const&) = delete; + + static VectPtr createVect(qproc::ChunkQuerySpec const& chunkQuerySpec, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& dbTablesMap); + + /// &&& doc + static VectPtr createVectFromJson(nlohmann::json const& ujJson, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& dbTablesMap); + + /// Return a json version of the contents of this class. + nlohmann::json serializeJson() const; + + std::vector const& getJobSubQueryTempIndexes() const { return _jobSubQueryTempIndexes; } + std::vector const& getJobDbTablesIndexes() const { return _jobDbTablesIndexes; } + std::vector const& getSubchunkIds() const { return _subchunkIds; } + + std::string dump() const; + +private: + JobFragment(JobSubQueryTempMap::Ptr const& subQueryTemplates, JobDbTablesMap::Ptr const& dbTablesMap); + + /// &&& doc + static void _addFragment(std::vector& jFragments, DbTableSet const& subChunkTables, + std::vector const& subchunkIds, std::vector const& queries, + JobSubQueryTempMap::Ptr const& subQueryTemplates, + JobDbTablesMap::Ptr const& dbTablesMap); + + JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< &&& doc + std::vector _jobSubQueryTempIndexes; ///< &&& doc + + JobDbTablesMap::Ptr _jobDbTablesMap; ///< &&& doc + std::vector _jobDbTablesIndexes; ///< &&& doc + + std::vector _subchunkIds; ///< &&& doc +}; + +/// This class is used to store the information for a single Job (the queries and metadata +/// required to collect rows from a single chunk) in a reasonable manner. +class JobMsg { +public: + using Ptr = std::shared_ptr; + using Vect = std::vector; + using VectPtr = std::shared_ptr; + std::string cName(const char* fnc) const { return std::string("JobMsg::") + fnc; } + + JobMsg() = delete; + JobMsg(JobMsg const&) = delete; + JobMsg& operator=(JobMsg const&) = delete; + + static Ptr create(std::shared_ptr const& jobs, + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& jobDbTablesMap); + + /// &&& doc + static Ptr createFromJson(nlohmann::json const& ujJson, JobSubQueryTempMap::Ptr const& subQueryTemplates, + JobDbTablesMap::Ptr const& dbTablesMap); + + /// Return a json version of the contents of this class. + nlohmann::json serializeJson() const; + + JobId getJobId() const { return _jobId; } + int getAttemptCount() const { return _attemptCount; } + std::string getChunkQuerySpecDb() const { return _chunkQuerySpecDb; } + int getScanRating() const { return _scanRating; } + bool getScanInteractive() const { return _scanInteractive; } + int getChunkId() const { return _chunkId; } + + std::vector const& getChunkScanTableIndexes() const { return _chunkScanTableIndexes; } + + JobFragment::VectPtr getJobFragments() const { return _jobFragments; } + +private: + JobMsg(std::shared_ptr const& jobPtr, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, + JobDbTablesMap::Ptr const& jobDbTablesMap); + + JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap, + JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int scanRating, + bool scanInteractive, int chunkId); + + JobId _jobId; + int _attemptCount; + std::string _chunkQuerySpecDb; + int _scanRating; + bool _scanInteractive; + int _chunkId; + JobFragment::VectPtr _jobFragments{new JobFragment::Vect()}; + + JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< Map of all query templates related to this UberJob. + JobDbTablesMap::Ptr _jobDbTablesMap; ///< Map of all db.tables related to this UberJob. + + std::vector _chunkScanTableIndexes; ///< list of indexes into _jobDbTablesMap. +}; + +/// This class stores an UberJob, a collection of Jobs meant for a +/// specific worker, so it can be converted to and from a json format +/// and sent to a worker. +/// There are several fields which are the same for each job, so these +/// values are stored in maps and the individual Jobs and Fragments +/// use integer indexes to reduce the size of the final message. +class UberJobMsg : public std::enable_shared_from_this { +public: + using Ptr = std::shared_ptr; + std::string cName(const char* fnc) const { return std::string("UberJobMsg::") + fnc; } + + UberJobMsg() = delete; + UberJobMsg(UberJobMsg const&) = delete; + UberJobMsg& operator=(UberJobMsg const&) = delete; + + static Ptr create(unsigned int metaVersion, std::string const& replicationInstanceId, + std::string const& replicationAuthKey, CzarContactInfo::Ptr const& czInfo, + WorkerContactInfo::Ptr const& wInfo, QueryId qId, UberJobId ujId, int rowLimit, + int maxTableSizeMB, ScanInfo::Ptr const& scanInfo_, + std::vector> const& jobs) { + return Ptr(new UberJobMsg(metaVersion, replicationInstanceId, replicationAuthKey, czInfo, wInfo->wId, + qId, ujId, rowLimit, maxTableSizeMB, scanInfo_, jobs)); + } + + static Ptr createFromJson(nlohmann::json const& ujJson); + + /// Return a json version of the contents of this class. + nlohmann::json serializeJson() const; + + QueryId getQueryId() const { return _qId; } + UberJobId getUberJobId() const { return _ujId; } + int getRowLimit() const { return _rowLimit; } + std::string getWorkerId() const { return _workerId; } + int getMaxTableSizeMb() const { return _maxTableSizeMB; } + + CzarContactInfo::Ptr getCzarContactInfo() const { return _czInfo; } + JobSubQueryTempMap::Ptr getJobSubQueryTempMap() const { return _jobSubQueryTempMap; } + JobDbTablesMap::Ptr getJobDbTablesMap() const { return _jobDbTablesMap; } + + JobMsg::VectPtr getJobMsgVect() const { return _jobMsgVect; } + + ScanInfo::Ptr getScanInfo() const { return _scanInfo; } + +private: + UberJobMsg(unsigned int metaVersion, std::string const& replicationInstanceId, + std::string const& replicationAuthKey, CzarContactInfo::Ptr const& czInfo, + std::string const& workerId, QueryId qId, UberJobId ujId, int rowLimit, int maxTableSizeMB, + ScanInfo::Ptr const& scanInfo_, std::vector> const& jobs); + + unsigned int _metaVersion; // "version", http::MetaModule::version + // czar + std::string _replicationInstanceId; // "instance_id", czarConfig->replicationInstanceId() + std::string _replicationAuthKey; //"auth_key", czarConfig->replicationAuthKey() + CzarContactInfo::Ptr _czInfo; + std::string _workerId; // "worker", ciwId + QueryId _qId; // "queryid", _queryId + UberJobId _ujId; // "uberjobid", _uberJobId + int _rowLimit; // "rowlimit", _rowLimit + int _maxTableSizeMB; // + + /// Map of all query templates related to this UberJob. + JobSubQueryTempMap::Ptr _jobSubQueryTempMap{JobSubQueryTempMap::create()}; + + /// Map of all db.tables related to this UberJob. + JobDbTablesMap::Ptr _jobDbTablesMap{JobDbTablesMap::create()}; + + /// List of all job data in this UberJob. "jobs", json::array() + JobMsg::VectPtr _jobMsgVect{new JobMsg::Vect()}; + + ScanInfo::Ptr _scanInfo{ScanInfo::create()}; ///< &&& doc +}; + +} // namespace lsst::qserv::protojson + +#endif // LSST_QSERV_PROTOJSON_UBERJOBMSG_H diff --git a/src/protojson/WorkerQueryStatusData.cc b/src/protojson/WorkerQueryStatusData.cc new file mode 100644 index 0000000000..ea3916b6fa --- /dev/null +++ b/src/protojson/WorkerQueryStatusData.cc @@ -0,0 +1,512 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "protojson/WorkerQueryStatusData.h" + +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "util/common.h" +#include "util/TimeUtils.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.WorkerQueryStatusData"); +} // namespace + +namespace lsst::qserv::protojson { + +json CzarContactInfo::serializeJson() const { + json jsCzar; + jsCzar["name"] = czName; + jsCzar["id"] = czId; + jsCzar["management-port"] = czPort; + jsCzar["management-host-name"] = czHostName; + jsCzar["czar-startup-time"] = czStartupTime; + return jsCzar; +} + +CzarContactInfo::Ptr CzarContactInfo::createFromJson(nlohmann::json const& czJson) { + try { + auto czName_ = http::RequestBodyJSON::required(czJson, "name"); + auto czId_ = http::RequestBodyJSON::required(czJson, "id"); + auto czPort_ = http::RequestBodyJSON::required(czJson, "management-port"); + auto czHostName_ = http::RequestBodyJSON::required(czJson, "management-host-name"); + auto czStartupTime_ = http::RequestBodyJSON::required(czJson, "czar-startup-time"); + return create(czName_, czId_, czPort_, czHostName_, czStartupTime_); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("CzarContactInfo::createJson invalid ") << exc.what()); + } + return nullptr; +} + +std::string CzarContactInfo::dump() const { + stringstream os; + os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << czHostName + << " czStartupTime=" << czStartupTime; + return os.str(); +} + +json WorkerContactInfo::serializeJson() const { + lock_guard lg(_rMtx); + return _serializeJson(); +} + +json WorkerContactInfo::_serializeJson() const { + json jsWorker; + jsWorker["id"] = wId; + jsWorker["host"] = _wHost; + jsWorker["management-host-name"] = _wManagementHost; + jsWorker["management-port"] = _wPort; + jsWorker["w-startup-time"] = _wStartupTime; + return jsWorker; +} + +WorkerContactInfo::Ptr WorkerContactInfo::createFromJsonRegistry(string const& wId_, + nlohmann::json const& regJson) { + try { + auto wHost_ = http::RequestBodyJSON::required(regJson, "host-addr"); + auto wManagementHost_ = http::RequestBodyJSON::required(regJson, "management-host-name"); + auto wPort_ = http::RequestBodyJSON::required(regJson, "management-port"); + auto updateTimeInt = http::RequestBodyJSON::required(regJson, "update-time-ms"); + TIMEPOINT updateTime_ = TIMEPOINT(chrono::milliseconds(updateTimeInt)); + + return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); + } + return nullptr; +} + +WorkerContactInfo::Ptr WorkerContactInfo::createFromJsonWorker(nlohmann::json const& wJson, + TIMEPOINT updateTime_) { + try { + auto wId_ = http::RequestBodyJSON::required(wJson, "id"); + auto wHost_ = http::RequestBodyJSON::required(wJson, "host"); + auto wManagementHost_ = http::RequestBodyJSON::required(wJson, "management-host-name"); + auto wPort_ = http::RequestBodyJSON::required(wJson, "management-port"); + + return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); + } + return nullptr; +} + +void WorkerContactInfo::setRegUpdateTime(TIMEPOINT updateTime) { + std::lock_guard lg(_rMtx); + _regUpdateTime = updateTime; + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " " << _dump()); +} + +string WorkerContactInfo::dump() const { + lock_guard lg(_rMtx); + return _dump(); +} + +string WorkerContactInfo::_dump() const { + stringstream os; + os << "workerContactInfo{" + << "id=" << wId << " host=" << _wHost << " mgHost=" << _wManagementHost << " port=" << _wPort + << " update=" << util::TimeUtils::timePointToDateTimeString(_regUpdateTime) << "}"; + return os.str(); +} + +shared_ptr WorkerQueryStatusData::serializeJson(double maxLifetime) { + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // message to send to the worker. + auto now = CLOCK::now(); + shared_ptr jsWorkerReqPtr = make_shared(); + json& jsWorkerR = *jsWorkerReqPtr; + jsWorkerR["version"] = http::MetaModule::version; + jsWorkerR["instance_id"] = _replicationInstanceId; + jsWorkerR["auth_key"] = _replicationAuthKey; + jsWorkerR["czarinfo"] = _czInfo->serializeJson(); + { + lock_guard lgI(_infoMtx); + if (_wInfo != nullptr) { + jsWorkerR["workerinfo"] = _wInfo->serializeJson(); + jsWorkerR["worker"] = _wInfo->wId; + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo is null"); + } + } + + // Note, old elements in the maps will be deleted after being added to the message + // to keep the czar from keeping track of these forever. + addListsToJson(jsWorkerR, now, maxLifetime); + if (czarCancelAfterRestart) { + jsWorkerR["czarrestart"] = true; + lock_guard mapLg(mapMtx); + jsWorkerR["czarrestartcancelczid"] = czarCancelAfterRestartCzId; + jsWorkerR["czarrestartcancelqid"] = czarCancelAfterRestartQId; + } else { + jsWorkerR["czarrestart"] = false; + } + + return jsWorkerReqPtr; +} + +void WorkerQueryStatusData::addListsToJson(json& jsWR, TIMEPOINT tmMark, double maxLifetime) { + jsWR["qiddonekeepfiles"] = json::array(); + jsWR["qiddonedeletefiles"] = json::array(); + jsWR["qiddeaduberjobs"] = json::array(); + lock_guard mapLg(mapMtx); + { + auto& jsDoneKeep = jsWR["qiddonekeepfiles"]; + auto iterDoneKeep = qIdDoneKeepFiles.begin(); + while (iterDoneKeep != qIdDoneKeepFiles.end()) { + auto qId = iterDoneKeep->first; + jsDoneKeep.push_back(qId); + auto tmTouched = iterDoneKeep->second; + double ageSecs = std::chrono::duration(tmMark - tmTouched).count(); + if (ageSecs > maxLifetime) { + iterDoneKeep = qIdDoneKeepFiles.erase(iterDoneKeep); + } else { + ++iterDoneKeep; + } + } + } + { + auto& jsDoneDelete = jsWR["qiddonedeletefiles"]; + auto iterDoneDelete = qIdDoneDeleteFiles.begin(); + while (iterDoneDelete != qIdDoneDeleteFiles.end()) { + auto qId = iterDoneDelete->first; + jsDoneDelete.push_back(qId); + auto tmStamp = iterDoneDelete->second; + double ageSecs = std::chrono::duration(tmMark - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterDoneDelete = qIdDoneDeleteFiles.erase(iterDoneDelete); + } else { + ++iterDoneDelete; + } + } + } + { + auto& jsDeadUj = jsWR["qiddeaduberjobs"]; + auto iterDeadUjQid = qIdDeadUberJobs.begin(); + while (iterDeadUjQid != qIdDeadUberJobs.end()) { + TIMEPOINT youngestTm = TIMEPOINT::max(); // need to find the youngest + auto qId = iterDeadUjQid->first; + auto& ujIdMap = iterDeadUjQid->second; + + json jsQidUj = {{"qid", qId}, {"ujids", json::array()}}; + auto& jsUjIds = jsQidUj["ujids"]; + + auto iterUjId = ujIdMap.begin(); + bool addedUjId = false; + + while (iterUjId != ujIdMap.end()) { + UberJobId ujId = iterUjId->first; + auto tmStamp = iterUjId->second; + if (tmStamp < youngestTm) { + youngestTm = tmStamp; + } + + jsUjIds.push_back(ujId); + addedUjId = true; + double ageSecs = std::chrono::duration(tmMark - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterUjId = ujIdMap.erase(iterUjId); + } else { + ++iterUjId; + } + } + + if (addedUjId) { + jsDeadUj.push_back(jsQidUj); + } + + // If the youngest element was too old, delete the map. + if (ujIdMap.empty() || std::chrono::duration(tmMark - youngestTm).count() > maxLifetime) { + iterDeadUjQid = qIdDeadUberJobs.erase(iterDeadUjQid); + } else { + ++iterDeadUjQid; + } + } + } +} + +WorkerQueryStatusData::Ptr WorkerQueryStatusData::createFromJson(nlohmann::json const& jsWorkerReq, + std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_, + TIMEPOINT updateTm) { + try { + if (jsWorkerReq["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson bad version"); + return nullptr; + } + + auto czInfo_ = CzarContactInfo::createFromJson(jsWorkerReq["czarinfo"]); + auto wInfo_ = WorkerContactInfo::createFromJsonWorker(jsWorkerReq["workerinfo"], updateTm); + if (czInfo_ == nullptr || wInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, + "WorkerQueryStatusData::createJson czar or worker info could not be parsed in " + << jsWorkerReq); + return nullptr; + } + auto wqsData = + WorkerQueryStatusData::create(wInfo_, czInfo_, replicationInstanceId_, replicationAuthKey_); + wqsData->parseLists(jsWorkerReq, updateTm); + + bool czarRestart = http::RequestBodyJSON::required(jsWorkerReq, "czarrestart"); + if (czarRestart) { + auto restartCzarId = + http::RequestBodyJSON::required(jsWorkerReq, "czarrestartcancelczid"); + auto restartQueryId = + http::RequestBodyJSON::required(jsWorkerReq, "czarrestartcancelqid"); + wqsData->setCzarCancelAfterRestart(restartCzarId, restartQueryId); + } + return wqsData; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); + } + return nullptr; +} + +void WorkerQueryStatusData::parseLists(nlohmann::json const& jsWR, TIMEPOINT updateTm) { + lock_guard mapLg(mapMtx); + parseListsInto(jsWR, updateTm, qIdDoneKeepFiles, qIdDoneDeleteFiles, qIdDeadUberJobs); +} + +void WorkerQueryStatusData::parseListsInto(nlohmann::json const& jsWR, TIMEPOINT updateTm, + std::map& doneKeepF, + std::map& doneDeleteF, + std::map>& deadUberJobs) { + auto& jsQIdDoneKeepFiles = jsWR["qiddonekeepfiles"]; + for (auto const& qidKeep : jsQIdDoneKeepFiles) { + doneKeepF[qidKeep] = updateTm; + } + + auto& jsQIdDoneDeleteFiles = jsWR["qiddonedeletefiles"]; + for (auto const& qidDelete : jsQIdDoneDeleteFiles) { + doneDeleteF[qidDelete] = updateTm; + } + + auto& jsQIdDeadUberJobs = jsWR["qiddeaduberjobs"]; + // Interestingly, !jsQIdDeadUberJobs.empty() doesn't work, but .size() > 0 does. + // Not having the size() check causes issues with the for loop trying to read the + // first element of an empty list, which goes badly. + if (jsQIdDeadUberJobs.size() > 0) { + for (auto const& qDeadUjs : jsQIdDeadUberJobs) { + QueryId qId = qDeadUjs["qid"]; + auto const& ujIds = qDeadUjs["ujids"]; + auto& mapOfUj = deadUberJobs[qId]; + for (auto const& ujId : ujIds) { + mapOfUj[ujId] = updateTm; + } + } + } +} + +void WorkerQueryStatusData::addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm) { + lock_guard mapLg(mapMtx); + auto& ujMap = qIdDeadUberJobs[qId]; + for (auto const ujId : ujIds) { + ujMap[ujId] = tm; + } +} + +void WorkerQueryStatusData::setWInfo(WorkerContactInfo::Ptr const& wInfo_) { + std::lock_guard lgI(_infoMtx); + if (_wInfo == nullptr) { + _wInfo = wInfo_; + return; + } + if (wInfo_ != nullptr) { + // This only changes host and port values of _wInfo. + _wInfo->changeBaseInfo(*wInfo_); + } + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " " << _wInfo->dump()); +} + +void WorkerQueryStatusData::addDeadUberJob(QueryId qId, UberJobId ujId, TIMEPOINT tm) { + lock_guard mapLg(mapMtx); + auto& ujMap = qIdDeadUberJobs[qId]; + ujMap[ujId] = tm; +} + +void WorkerQueryStatusData::addToDoneDeleteFiles(QueryId qId) { + lock_guard mapLg(mapMtx); + qIdDoneDeleteFiles[qId] = CLOCK::now(); +} + +void WorkerQueryStatusData::addToDoneKeepFiles(QueryId qId) { + lock_guard mapLg(mapMtx); + qIdDoneKeepFiles[qId] = CLOCK::now(); +} + +void WorkerQueryStatusData::removeDeadUberJobsFor(QueryId qId) { + lock_guard mapLg(mapMtx); + qIdDeadUberJobs.erase(qId); +} + +json WorkerQueryStatusData::serializeResponseJson(uint64_t workerStartupTime) { + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // response. Nothing should be deleted and time is irrelevant for this, so maxLifetime is enormous + // and any time could be used for last contact, but now() is easy. + // This is only called by the worker. As such nothing should be deleted here as the lifetime of + // these elements is determined by the lifetime of the owning UserQueryInfo instance. + double maxLifetime = std::numeric_limits::max(); + auto now = CLOCK::now(); + json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; + jsResp["w-startup-time"] = workerStartupTime; + addListsToJson(jsResp, now, maxLifetime); + return jsResp; +} + +bool WorkerQueryStatusData::handleResponseJson(nlohmann::json const& jsResp) { + auto now = CLOCK::now(); + std::map doneKeepF; + std::map doneDeleteF; + std::map> deadUberJobs; + parseListsInto(jsResp, now, doneKeepF, doneDeleteF, deadUberJobs); + + lock_guard mapLg(mapMtx); + // Remove entries from _qIdDoneKeepFiles + for (auto const& [qId, tm] : doneKeepF) { + qIdDoneKeepFiles.erase(qId); + } + + // Remove entries from _qIdDoneDeleteFiles + for (auto const& [qId, tm] : doneDeleteF) { + qIdDoneDeleteFiles.erase(qId); + } + + // Remove entries from _qIdDeadUberJobs + for (auto const& [qId, ujMap] : deadUberJobs) { + auto iter = qIdDeadUberJobs.find(qId); + if (iter != qIdDeadUberJobs.end()) { + auto& deadMap = iter->second; + for (auto const& [ujId, tm] : ujMap) { + deadMap.erase(ujId); + } + if (deadMap.empty()) { + qIdDeadUberJobs.erase(iter); + } + } + } + + bool workerRestarted = false; + auto workerStartupTime = http::RequestBodyJSON::required(jsResp, "w-startup-time"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " workerStartupTime=" << workerStartupTime); + if (!_wInfo->checkWStartupTime(workerStartupTime)) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " startup time for worker=" << _wInfo->dump() + << " changed to=" << workerStartupTime << " Assuming worker restarted"); + workerRestarted = true; + } + return workerRestarted; +} + +string WorkerQueryStatusData::dump() const { + lock_guard lgI(_infoMtx); + return _dump(); +} + +string WorkerQueryStatusData::_dump() const { + VMUTEX_HELD(_infoMtx); + stringstream os; + os << "ActiveWorker " << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); + return os.str(); +} + +shared_ptr WorkerCzarComIssue::serializeJson() { + shared_ptr jsCzarReqPtr = make_shared(); + json& jsCzarR = *jsCzarReqPtr; + lock_guard _lgWciMtx(_wciMtx); + if (_wInfo == nullptr || _czInfo == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _wInfo or _czInfo was null"); + return jsCzarReqPtr; + } + + jsCzarR["version"] = http::MetaModule::version; + jsCzarR["instance_id"] = _replicationInstanceId; + jsCzarR["auth_key"] = _replicationAuthKey; + jsCzarR["czarinfo"] = _czInfo->serializeJson(); + jsCzarR["czar"] = _czInfo->czName; + jsCzarR["workerinfo"] = _wInfo->serializeJson(); + + jsCzarR["thoughtczarwasdead"] = _thoughtCzarWasDead; + + // TODO:UJ add list of failed transmits + + return jsCzarReqPtr; +} + +WorkerCzarComIssue::Ptr WorkerCzarComIssue::createFromJson(nlohmann::json const& jsCzarReq, + std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_) { + string const fName("WorkerCzarComIssue::createFromJson"); + LOGS(_log, LOG_LVL_DEBUG, fName); + try { + if (jsCzarReq["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, fName << " bad version"); + return nullptr; + } + + auto czInfo_ = CzarContactInfo::createFromJson(jsCzarReq["czarinfo"]); + auto now = CLOCK::now(); + auto wInfo_ = WorkerContactInfo::createFromJsonWorker(jsCzarReq["workerinfo"], now); + if (czInfo_ == nullptr || wInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, fName << " or worker info could not be parsed in " << jsCzarReq); + } + auto wccIssue = create(replicationInstanceId_, replicationAuthKey_); + wccIssue->setContactInfo(wInfo_, czInfo_); + wccIssue->_thoughtCzarWasDead = + http::RequestBodyJSON::required(jsCzarReq, "thoughtczarwasdead"); + return wccIssue; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); + } + return nullptr; +} + +json WorkerCzarComIssue::serializeResponseJson() { + json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; + + // TODO:UJ add lists of uberjobs that are scheduled to have files collected because of this message. + return jsResp; +} + +string WorkerCzarComIssue::dump() const { + lock_guard _lgWciMtx(_wciMtx); + return _dump(); +} + +string WorkerCzarComIssue::_dump() const { + stringstream os; + os << "WorkerCzarComIssue wInfo=" << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); + os << " czInfo=" << _czInfo->dump(); + os << " thoughtCzarWasDead=" << _thoughtCzarWasDead; + return os.str(); +} + +} // namespace lsst::qserv::protojson diff --git a/src/protojson/WorkerQueryStatusData.h b/src/protojson/WorkerQueryStatusData.h new file mode 100644 index 0000000000..73aebe2449 --- /dev/null +++ b/src/protojson/WorkerQueryStatusData.h @@ -0,0 +1,478 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_PROTOJSON_WORKERQUERYSTATUSDATA_H +#define LSST_QSERV_PROTOJSON_WORKERQUERYSTATUSDATA_H + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "global/clock_defs.h" +#include "global/intTypes.h" +#include "util/Mutex.h" + +// This header declarations +namespace lsst::qserv::protojson { + +/// This class just contains the czar id and network contact information. +class CzarContactInfo : public std::enable_shared_from_this { +public: + using Ptr = std::shared_ptr; + std::string cName(const char* fnc) const { return std::string("CzarContactInfo") + fnc; } + + CzarContactInfo() = delete; + CzarContactInfo(CzarContactInfo const&) = default; + CzarContactInfo& operator=(CzarContactInfo const&) = default; + + /// Return true is elements, other than czStartupTime, are the same. + bool compare(CzarContactInfo const& other) { + return (czName == other.czName && czId == other.czId && czPort == other.czPort && + czHostName == other.czHostName); + } + + static Ptr create(std::string const& czName_, CzarIdType czId_, int czPort_, + std::string const& czHostName_, uint64_t czStartupTime_) { + return Ptr(new CzarContactInfo(czName_, czId_, czPort_, czHostName_, czStartupTime_)); + } + + static Ptr createFromJson(nlohmann::json const& czarJson); + + std::string const czName; ///< czar "name" + CzarIdType const czId; ///< czar "id" + int const czPort; ///< czar "management-port" + std::string const czHostName; ///< czar "management-host-name" + uint64_t const czStartupTime; ///< czar startup time + + /// Return a json version of the contents of this class. + nlohmann::json serializeJson() const; + + std::string dump() const; + +private: + CzarContactInfo(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_, + uint64_t czStartupTime_) + : czName(czName_), + czId(czId_), + czPort(czPort_), + czHostName(czHostName_), + czStartupTime(czStartupTime_) {} +}; + +/// This class just contains the worker id and network communication information. +class WorkerContactInfo { +public: + using Ptr = std::shared_ptr; + + using WCMap = std::unordered_map; + using WCMapPtr = std::shared_ptr; + + static Ptr create(std::string const& wId_, std::string const& wHost_, std::string const& wManagementHost_, + int wPort_, TIMEPOINT updateTime_) { + return Ptr(new WorkerContactInfo(wId_, wHost_, wManagementHost_, wPort_, updateTime_)); + } + + /// This function creates a WorkerQueryStatusData object from a registry json message, + /// which is provided by the system registry. + static Ptr createFromJsonRegistry(std::string const& wId_, nlohmann::json const& regJson); + + /// This function creates a WorkerQueryStatusData object from a worker json message. + static Ptr createFromJsonWorker(nlohmann::json const& workerJson, TIMEPOINT updateTime); + + /// Return a json version of the contents of this object. + nlohmann::json serializeJson() const; + + std::string cName(const char* fn) { return std::string("WorkerContactInfo::") + fn; } + + std::string const wId; ///< key, this is the one thing that cannot change. + + std::string getWHost() const { + std::lock_guard lg(_rMtx); + return _wHost; + } + + std::string getWManagementHost() const { + std::lock_guard lg(_rMtx); + return _wManagementHost; + } + + int getWPort() const { + std::lock_guard lg(_rMtx); + return _wPort; + } + + /// Change host and port info to those provided in `other`. + void changeBaseInfo(WorkerContactInfo const& other) { + auto [oWId, oWHost, oWManagementHost, oWPort] = other.getAll(); + std::lock_guard lg(_rMtx); + _wHost = oWHost; + _wManagementHost = oWManagementHost; + _wPort = oWPort; + } + + /// @return wId - workerId + /// @return _wHost - worker host + /// @return _wManagementHost - management host + /// @return _wPort - worker port + std::tuple getAll() const { + std::lock_guard lg(_rMtx); + return {wId, _wHost, _wManagementHost, _wPort}; + } + + /// Return true if communication related items are the same. + bool isSameContactInfo(WorkerContactInfo const& other) const { + auto [oWId, oWHost, oWManagementHost, oWPort] = other.getAll(); + std::lock_guard lg(_rMtx); + return (wId == oWId && _wHost == oWHost && _wManagementHost == oWManagementHost && _wPort == oWPort); + } + + void setRegUpdateTime(TIMEPOINT updateTime); + + TIMEPOINT getRegUpdateTime(TIMEPOINT updateTime) { + std::lock_guard lg(_rMtx); + return _regUpdateTime; + } + + double timeSinceRegUpdateSeconds() const { + std::lock_guard lg(_rMtx); + double secs = std::chrono::duration(CLOCK::now() - _regUpdateTime).count(); + return secs; + } + + TIMEPOINT getRegUpdateTime() const { + std::lock_guard lg(_rMtx); + return _regUpdateTime; + } + + /// @return true if startupTime equals _wStartupTime or _wStartupTime was never set, + /// if _wStartupTime was never set, it is set to startupTime. + /// @return false indicates the worker was restarted and all associated jobs need + /// re-assignment. + bool checkWStartupTime(uint64_t startupTime) { + std::lock_guard lg(_rMtx); + if (_wStartupTime == startupTime) { + return true; + } + if (_wStartupTime == 0) { + _wStartupTime = startupTime; + return true; + } + _wStartupTime = startupTime; + return false; + } + + uint64_t getWStartupTime() const { + std::lock_guard lg(_rMtx); + return _wStartupTime; + } + + std::string dump() const; + +private: + WorkerContactInfo(std::string const& wId_, std::string const& wHost_, std::string const& wManagementHost_, + int wPort_, TIMEPOINT updateTime_) + : wId(wId_), _wHost(wHost_), _wManagementHost(wManagementHost_), _wPort(wPort_) { + setRegUpdateTime(updateTime_); + } + + // _rMtx must be locked before calling + std::string _dump() const; + + // _rMtx must be locked before calling + nlohmann::json _serializeJson() const; + + std::string _wHost; ///< "host-addr" entry. + std::string _wManagementHost; ///< "management-host-name" entry. + int _wPort; ///< "management-port" entry. + + /// Last time the registry heard from this worker. The ActiveWorker class + /// will use this to determine the worker's state (alive/dead). + TIMEPOINT _regUpdateTime; + + /// "w-startup-time", it's value is set to zero until the real value is + /// received from the worker. Once it is non-zero, any change indicates + /// the worker was restarted and all UberJobs that were assigned there + /// need to be unassigned. On the worker, this should always be set from + /// foreman()->getStartupTime(); + uint64_t _wStartupTime = 0; + + mutable MUTEX _rMtx; ///< protects _regUpdate +}; + +/// This classes purpose is to be a structure to store and transfer information +/// about which queries have been completed or cancelled on the worker. This +/// class contains the functions that encode and decode the data they contain +/// to and from a json format. +class WorkerQueryStatusData { +public: + using Ptr = std::shared_ptr; + + WorkerQueryStatusData() = delete; + WorkerQueryStatusData(WorkerQueryStatusData const&) = delete; + WorkerQueryStatusData& operator=(WorkerQueryStatusData const&) = delete; + + std::string cName(const char* fName) { return std::string("WorkerQueryStatusData::") + fName; } + + static Ptr create(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_, + std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { + return Ptr(new WorkerQueryStatusData(wInfo_, czInfo_, replicationInstanceId_, replicationAuthKey_)); + } + + /// This function creates a WorkerQueryStatusData object from the worker json `czarJson`, the + /// other parameters are used to verify the json message. + static Ptr createFromJson(nlohmann::json const& czarJson, std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_, TIMEPOINT updateTm); + + ~WorkerQueryStatusData() = default; + + void setWInfo(WorkerContactInfo::Ptr const& wInfo_); + + WorkerContactInfo::Ptr getWInfo() const { + std::lock_guard lgI(_infoMtx); + return _wInfo; + } + CzarContactInfo::Ptr getCzInfo() const { return _czInfo; } + + /// `qId` and `ujId` identify a dead UberJob which is added to the list + /// of dead UberJobs for this worker. + void addDeadUberJob(QueryId qId, UberJobId ujId, TIMEPOINT tm); + + /// Add multiple UberJobIds for `qId` to the list of dead UberJobs for + /// this worker. + void addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm); + + /// Add `qId` to the list of user queries where all Tasks can be stopped + /// and result files can be deleted. + void addToDoneDeleteFiles(QueryId qId); + + /// Add `qId` to the list of user queries where all Tasks can be stopped + /// but result files should be kept. + void addToDoneKeepFiles(QueryId qId); + + /// Remove all UberJobs from the list of dead UberJobs with QueryId `qId`. + /// There's no point in tracking individual UberJobs once the entire + /// user query is finished or cancelled as they will all be deleted by + /// `addToDoneDeleteFiles` + void removeDeadUberJobsFor(QueryId qId); + + void setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { + std::lock_guard mapLg(mapMtx); + czarCancelAfterRestart = true; + czarCancelAfterRestartCzId = czId; + czarCancelAfterRestartQId = lastQId; + } + + bool isCzarRestart() const { return czarCancelAfterRestart; } + CzarIdType getCzarRestartCzarId() const { return czarCancelAfterRestartCzId; } + QueryId getCzarRestartQueryId() const { return czarCancelAfterRestartQId; } + + /// Create a json object held by a shared pointer to use as a message. + /// Old objects in this instance will be removed after being added to the + /// json message. + std::shared_ptr serializeJson(double maxLifetime); + + /// Add contents of qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs to `jsWR`, + /// and remove map elements that have an age (tmMark - element.touchTime) greater + /// than maxLifetime. + void addListsToJson(nlohmann::json& jsWR, TIMEPOINT tmMark, double maxLifetime); + + /// Parse the lists in `jsWR` to populate the lists for qIdDoneKeepFiles, + /// qIdDoneDeleteFiles, and qIdDeadUberJobs. + /// @throws std::invalid_argument + void parseLists(nlohmann::json const& jsWR, TIMEPOINT updateTm); + + /// Return a json object indicating the status of the message for the + /// original requester. + nlohmann::json serializeResponseJson(uint64_t workerStartupTime); + + /// Use the worker's response, `jsResp`, to update the status of this object. + /// The worker's response contains lists indicating what the worker + /// received from the czar's json message created with `serializeResponseJson`. + /// The czar can remove the ids from the lists as once the worker has + /// verified them. + /// @return transmitSuccess - true if the message was parsed successfully. + /// @return workerRestarted - true if `workerStartupTime` doesn't match, + /// indicating the worker has been restarted and the czar should + /// invalidate and re-assign all UberJobs associated with this + /// worker. + /// @throw invalid_argument if there are problems with json parsing. + bool handleResponseJson(nlohmann::json const& jsResp); + + /// Parse the contents of `jsWR` to fill the maps `doneKeepF`, `doneDeleteF`, + /// and `deadUberJobs`. + static void parseListsInto(nlohmann::json const& jsWR, TIMEPOINT updateTm, + std::map& doneKeepF, + std::map& doneDeleteF, + std::map>& deadUberJobs); + + std::string dump() const; + + // Making these private requires member functions to be written + // that cause issues with linking. All of the workarounds are ugly. + /// Map of QueryIds where the LIMIT clause has been satisfied so + /// that Tasks can be stopped but result files need to be kept. + std::map qIdDoneKeepFiles; + + /// Map fo QueryIds where Tasks can be stopped and files deleted, which is + /// used when user queries are cancelled or finished. + std::map qIdDoneDeleteFiles; + + /// Map used to indicated a specific UberJobs need to be killed. + std::map> qIdDeadUberJobs; + + /// If true, this indicates that this is a newly started czar and + /// the worker should stop all previous work associated with this + /// CzarId. + std::atomic czarCancelAfterRestart = false; + CzarIdType czarCancelAfterRestartCzId = 0; + QueryId czarCancelAfterRestartQId = 0; + + /// Protects _qIdDoneKeepFiles, _qIdDoneDeleteFiles, _qIdDeadUberJobs, + /// and czarCancelAfter variables. + mutable MUTEX mapMtx; + +private: + WorkerQueryStatusData(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_, + std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) + : _wInfo(wInfo_), + _czInfo(czInfo_), + _replicationInstanceId(replicationInstanceId_), + _replicationAuthKey(replicationAuthKey_) {} + + WorkerContactInfo::Ptr _wInfo; ///< Information needed to contact the worker. + CzarContactInfo::Ptr const _czInfo; ///< Information needed to contact the czar. + mutable MUTEX _infoMtx; ///< protects _wInfo + + std::string const _replicationInstanceId; ///< Used for message verification. + std::string const _replicationAuthKey; ///< Used for message verification. + + /// _infoMtx must be locked before calling. + std::string _dump() const; +}; + +/// This class is used to send/receive a message from the worker to a specific +/// czar when there has been a communication issue with the worker sending UberJob +/// file ready messages. If there have been timeouts, the worker will send this +/// message to the czar immediately after the worker receives a +/// WorkerQueryStatusData message from the czar (indicating that communication +/// is now possible). +/// If communication with the czar has failed for a long time, the worker +/// will set "_thoughtCzarWasDead" and delete all incomplete work associated +/// with that czar. Result files will remain until garbage cleanup or the czar +/// calls for their removal. +/// TODO:UJ &&& UberJob complete messages that failed to be sent to the czar +/// TODO:UJ &&& will be added to this message. uber job file response +/// Upon successful completion, the worker will clear all values set by the +/// the czar. +/// Currently, this message is expected to only be needed rarely. +class WorkerCzarComIssue { +public: + using Ptr = std::shared_ptr; + + WorkerCzarComIssue() = delete; + ~WorkerCzarComIssue() = default; + + std::string cName(const char* funcN) { return std::string("WorkerCzarComIssue") + funcN; } + + static Ptr create(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { + return Ptr(new WorkerCzarComIssue(replicationInstanceId_, replicationAuthKey_)); + } + + static Ptr createFromJson(nlohmann::json const& workerJson, std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_); + + void setThoughtCzarWasDead(bool wasDead) { + std::lock_guard lg(_wciMtx); + _thoughtCzarWasDead = wasDead; + } + + bool getThoughtCzarWasDead() const { return _thoughtCzarWasDead; } + + /// Return true if there is a reason this WorkerCzarComIssue should be sent to this czar. + bool needToSend() const { + std::lock_guard lg(_wciMtx); + // TODO:UJ &&& or list of failed transmits not empty. + return _thoughtCzarWasDead; + } + + /// Set the contact information for the appropriate czar and worker. + void setContactInfo(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_) { + std::lock_guard lgWci(_wciMtx); + if (_wInfo == nullptr && wInfo_ != nullptr) _wInfo = wInfo_; + if (_czInfo == nullptr && czInfo_ != nullptr) _czInfo = czInfo_; + } + + CzarContactInfo::Ptr getCzarInfo() const { + std::lock_guard lgWci(_wciMtx); + return _czInfo; + } + + WorkerContactInfo::Ptr getWorkerInfo() const { + std::lock_guard lgWci(_wciMtx); + return _wInfo; + } + + /// Return a json version of the contents of this class. + std::shared_ptr serializeJson(); + + /// Return a json object indicating the status of the message for the + /// original requester. + nlohmann::json serializeResponseJson(); + + std::string dump() const; + +private: + WorkerCzarComIssue(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) + : _replicationInstanceId(replicationInstanceId_), _replicationAuthKey(replicationAuthKey_) {} + + std::string _dump() const; + + WorkerContactInfo::Ptr _wInfo; + CzarContactInfo::Ptr _czInfo; + std::string const _replicationInstanceId; ///< Used for message verification. + std::string const _replicationAuthKey; ///< Used for message verification. + + /// Set to by the worker true if the czar was considered dead, and reset to false + /// after the czar has acknowledged successful reception of this message. + bool _thoughtCzarWasDead = false; + + mutable MUTEX _wciMtx; ///< protects all members. +}; + +class WorkerUberJobMsg { +public: + using Ptr = std::shared_ptr; + + static Ptr create(); + +private: + WorkerUberJobMsg(); +}; + +} // namespace lsst::qserv::protojson + +#endif // LSST_QSERV_PROTOJSON_WORKERQUERYSTATUSDATA_H diff --git a/src/protojson/testStatusData.cc b/src/protojson/testStatusData.cc new file mode 100644 index 0000000000..8dd226080d --- /dev/null +++ b/src/protojson/testStatusData.cc @@ -0,0 +1,181 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include +#include +#include + +// Qserv headers +#include "global/clock_defs.h" +#include "lsst/log/Log.h" +#include "protojson/WorkerQueryStatusData.h" + +// Boost unit test header +#define BOOST_TEST_MODULE RequestQuery +#include + +using namespace std; +namespace test = boost::test_tools; +using namespace lsst::qserv::protojson; + +BOOST_AUTO_TEST_SUITE(Suite) + +BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { + string const replicationInstanceId = "repliInstId"; + string const replicationAuthKey = "repliIAuthKey"; + + uint64_t cxrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 5s); + uint64_t wkrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 10s); + + string const czrName("czar_name"); + lsst::qserv::CzarIdType const czrId = 32; + int czrPort = 2022; + string const czrHost("cz_host"); + + auto czarA = + lsst::qserv::protojson::CzarContactInfo::create(czrName, czrId, czrPort, czrHost, cxrStartTime); + + auto czarAJs = czarA->serializeJson(); + + auto czarB = lsst::qserv::protojson::CzarContactInfo::createFromJson(czarAJs); + BOOST_REQUIRE(czarA->compare(*czarB)); + + auto czarC = lsst::qserv::protojson::CzarContactInfo::create("different", czrId, czrPort, czrHost, + cxrStartTime); + BOOST_REQUIRE(!czarA->compare(*czarC)); + + auto start = lsst::qserv::CLOCK::now(); + auto workerA = WorkerContactInfo::create("sd_workerA", "host_w1", "mgmhost_a", 3421, start); + + auto workerB = WorkerContactInfo::create("sd_workerB", "host_w2", "mgmhost_a", 3421, start); + auto workerC = WorkerContactInfo::create("sd_workerC", "host_w3", "mgmhost_b", 3422, start); + + auto jsWorkerA = workerA->serializeJson(); + auto start1Sec = start + 1s; + auto workerA1 = WorkerContactInfo::createFromJsonWorker(jsWorkerA, start1Sec); + BOOST_REQUIRE(workerA->isSameContactInfo(*workerA1)); + + // WorkerQueryStatusData + auto wqsdA = lsst::qserv::protojson::WorkerQueryStatusData::create(workerA, czarA, replicationInstanceId, + replicationAuthKey); + + double maxLifetime = 300.0; + auto jsDataA = wqsdA->serializeJson(maxLifetime); + + // Check that empty lists work. + auto wqsdA1 = lsst::qserv::protojson::WorkerQueryStatusData::createFromJson( + *jsDataA, replicationInstanceId, replicationAuthKey, start1Sec); + auto jsDataA1 = wqsdA1->serializeJson(maxLifetime); + BOOST_REQUIRE(*jsDataA == *jsDataA1); + + vector qIdsDelFiles = {7, 8, 9, 15, 25, 26, 27, 30}; + vector qIdsKeepFiles = {1, 2, 3, 4, 6, 10, 13, 19, 33}; + for (auto const qIdDF : qIdsDelFiles) { + wqsdA->qIdDoneDeleteFiles[qIdDF] = start; + } + + jsDataA = wqsdA->serializeJson(maxLifetime); + BOOST_REQUIRE(*jsDataA != *jsDataA1); + + for (auto const qIdKF : qIdsKeepFiles) { + wqsdA->qIdDoneKeepFiles[qIdKF] = start; + } + + wqsdA->addDeadUberJobs(12, {1, 3}, start); + + jsDataA = wqsdA->serializeJson(maxLifetime); + + auto start5Sec = start + 5s; + auto workerAFromJson = lsst::qserv::protojson::WorkerQueryStatusData::createFromJson( + *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); + auto jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); + BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); + + wqsdA->addDeadUberJobs(12, {34}, start5Sec); + wqsdA->addDeadUberJobs(91, {77}, start5Sec); + wqsdA->addDeadUberJobs(1059, {1, 4, 6, 7, 8, 10, 3, 22, 93}, start5Sec); + + jsDataA = wqsdA->serializeJson(maxLifetime); + BOOST_REQUIRE(*jsDataA != *jsWorkerAFromJson); + + workerAFromJson = lsst::qserv::protojson::WorkerQueryStatusData::createFromJson( + *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); + jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); + BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); + + // Make the response, which contains lists of the items handled by the workers. + auto jsWorkerResp = workerAFromJson->serializeResponseJson(wkrStartTime); + + // test removal of elements after response. + BOOST_REQUIRE(!wqsdA->qIdDoneDeleteFiles.empty()); + BOOST_REQUIRE(!wqsdA->qIdDoneKeepFiles.empty()); + BOOST_REQUIRE(!wqsdA->qIdDeadUberJobs.empty()); + + wqsdA->handleResponseJson(jsWorkerResp); + auto workerRestarted = wqsdA->handleResponseJson(jsWorkerResp); + BOOST_REQUIRE(workerRestarted == false); + + BOOST_REQUIRE(wqsdA->qIdDoneDeleteFiles.empty()); + BOOST_REQUIRE(wqsdA->qIdDoneKeepFiles.empty()); + BOOST_REQUIRE(wqsdA->qIdDeadUberJobs.empty()); +} + +BOOST_AUTO_TEST_CASE(WorkerCzarComIssue) { + string const replicationInstanceId = "repliInstId"; + string const replicationAuthKey = "repliIAuthKey"; + + uint64_t cxrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 5s); + + string const czrName("czar_name"); + lsst::qserv::CzarIdType const czrId = 32; + int czrPort = 2022; + string const czrHost("cz_host"); + + auto czarA = + lsst::qserv::protojson::CzarContactInfo::create(czrName, czrId, czrPort, czrHost, cxrStartTime); + auto czarAJs = czarA->serializeJson(); + + auto start = lsst::qserv::CLOCK::now(); + auto workerA = WorkerContactInfo::create("sd_workerA", "host_w1", "mgmhost_a", 3421, start); + auto jsWorkerA = workerA->serializeJson(); + + // WorkerCzarComIssue + auto wccIssueA = + lsst::qserv::protojson::WorkerCzarComIssue::create(replicationInstanceId, replicationAuthKey); + wccIssueA->setContactInfo(workerA, czarA); + BOOST_REQUIRE(wccIssueA->needToSend() == false); + wccIssueA->setThoughtCzarWasDead(true); + BOOST_REQUIRE(wccIssueA->needToSend() == true); + + auto jsIssueA = wccIssueA->serializeJson(); + + auto wccIssueA1 = lsst::qserv::protojson::WorkerCzarComIssue::createFromJson( + *jsIssueA, replicationInstanceId, replicationAuthKey); + auto jsIssueA1 = wccIssueA1->serializeJson(); + BOOST_REQUIRE(*jsIssueA == *jsIssueA1); + + // TODO:UJ Test with items in lists. +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/protojson/testUberJobMsg.cc b/src/protojson/testUberJobMsg.cc new file mode 100644 index 0000000000..32412e8658 --- /dev/null +++ b/src/protojson/testUberJobMsg.cc @@ -0,0 +1,130 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include +#include +#include + +#include "nlohmann/json.hpp" + +// Qserv headers +#include "global/clock_defs.h" +#include "lsst/log/Log.h" +#include "protojson/UberJobMsg.h" + +// Boost unit test header +#define BOOST_TEST_MODULE RequestQuery +#include + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.testUberJobMsg"); +} + +using namespace std; +namespace test = boost::test_tools; +using namespace lsst::qserv::protojson; + +BOOST_AUTO_TEST_SUITE(Suite) +#if 0 //&&& +std::string testA() { + std::string ta = + R"({"maxtablesizemb":5432,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter` AS `qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; + return ta; +} +#endif // &&& + +string testA() { + string ta = + R"({"maxtablesizemb":5432,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; + return ta; +} + +string testB() { + string tb = + R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":39,"worker":"db04"})"; + return tb; +} + +bool parseSerializeReparseCheck(string const& jsStr, string const& note) { + string fName("parseSerialize "); + fName += note + " "; + LOGS(_log, LOG_LVL_INFO, fName << " start " << jsStr); + nlohmann::json js = nlohmann::json::parse(jsStr); + LOGS(_log, LOG_LVL_INFO, fName << " parse 1"); + + UberJobMsg::Ptr ujm = UberJobMsg::createFromJson(js); + BOOST_REQUIRE(ujm != nullptr); + + nlohmann::json jsUjm = ujm->serializeJson(); + LOGS(_log, LOG_LVL_INFO, fName << " serialized jsUjm=" << jsUjm); + + UberJobMsg::Ptr ujmCreated = UberJobMsg::createFromJson(jsUjm); + LOGS(_log, LOG_LVL_INFO, fName << " created"); + nlohmann::json jsUjmCreated = ujmCreated->serializeJson(); + LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); + + bool createdMatchesOriginal = jsUjm == jsUjmCreated; + if (createdMatchesOriginal) { + LOGS(_log, LOG_LVL_INFO, fName << "created matches original"); + } else { + LOGS(_log, LOG_LVL_ERROR, "jsUjm != jsUjmCreated"); + LOGS(_log, LOG_LVL_ERROR, "jsUjm=" << jsUjm); + LOGS(_log, LOG_LVL_ERROR, "jsUjmCreated=" << jsUjmCreated); + } + BOOST_REQUIRE(createdMatchesOriginal); + return createdMatchesOriginal; +} + +BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { + string const replicationInstanceId = "repliInstId"; + string const replicationAuthKey = "repliIAuthKey"; + + LOGS(_log, LOG_LVL_INFO, "testUJM start"); + string jsStr = testA(); + nlohmann::json js = nlohmann::json::parse(jsStr); + UberJobMsg::Ptr ujm = UberJobMsg::createFromJson(js); + BOOST_REQUIRE(ujm != nullptr); + + nlohmann::json jsUjm = ujm->serializeJson(); + + LOGS(_log, LOG_LVL_INFO, "js=" << js); + LOGS(_log, LOG_LVL_INFO, "jsUjm=" << jsUjm); + + UberJobMsg::Ptr ujmCreated = UberJobMsg::createFromJson(jsUjm); + LOGS(_log, LOG_LVL_INFO, "ujmCreated=" << ujmCreated); + nlohmann::json jsUjmCreated = ujmCreated->serializeJson(); + + bool createdMatchesOriginal = jsUjm == jsUjmCreated; + if (!createdMatchesOriginal) { + LOGS(_log, LOG_LVL_ERROR, "jsUjm != jsUjmCreated"); + LOGS(_log, LOG_LVL_ERROR, "jsUjm=" << jsUjm); + LOGS(_log, LOG_LVL_ERROR, "jsUjmCreated=" << jsUjmCreated); + } + BOOST_REQUIRE(createdMatchesOriginal); + + BOOST_REQUIRE(parseSerializeReparseCheck(testA(), "A")); + BOOST_REQUIRE(parseSerializeReparseCheck(testB(), "B")); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/qana/CMakeLists.txt b/src/qana/CMakeLists.txt index d59c10bf0c..30deb59caa 100644 --- a/src/qana/CMakeLists.txt +++ b/src/qana/CMakeLists.txt @@ -40,7 +40,6 @@ FUNCTION(qana_tests) css qmeta rproc - xrdreq Boost::unit_test_framework Threads::Threads ) diff --git a/src/qana/QueryMapping.h b/src/qana/QueryMapping.h index 2e8dca319b..585971f976 100644 --- a/src/qana/QueryMapping.h +++ b/src/qana/QueryMapping.h @@ -92,6 +92,8 @@ class QueryMapping { bool hasParameter(Parameter p) const; DbTableSet const& getSubChunkTables() const { return _subChunkTables; } + std::string dump() const { return std::string("&&& NEED CODE"); } + private: ParameterMap _subs; DbTableSet _subChunkTables; diff --git a/src/qana/ScanTablePlugin.cc b/src/qana/ScanTablePlugin.cc index 8c3fcde007..cf1ecc4dfb 100644 --- a/src/qana/ScanTablePlugin.cc +++ b/src/qana/ScanTablePlugin.cc @@ -42,7 +42,6 @@ // Qserv headers #include "czar/Czar.h" #include "global/stringTypes.h" -#include "proto/ScanTableInfo.h" #include "query/ColumnRef.h" #include "query/FromList.h" #include "query/QueryContext.h" @@ -67,8 +66,8 @@ void ScanTablePlugin::applyLogical(query::SelectStmt& stmt, query::QueryContext& void ScanTablePlugin::applyFinal(query::QueryContext& context) { int const scanThreshold = _interactiveChunkLimit; if (context.chunkCount < scanThreshold) { - context.scanInfo.infoTables.clear(); - context.scanInfo.scanRating = 0; + context.scanInfo->infoTables.clear(); + context.scanInfo->scanRating = 0; LOGS(_log, LOG_LVL_INFO, "ScanInfo Squash full table scan tables: <" << scanThreshold << " chunks."); } } @@ -95,7 +94,8 @@ StringPairVector filterPartitioned(query::TableRefList const& tList) { return vector; } -proto::ScanInfo ScanTablePlugin::_findScanTables(query::SelectStmt& stmt, query::QueryContext& context) { +protojson::ScanInfo::Ptr ScanTablePlugin::_findScanTables(query::SelectStmt& stmt, + query::QueryContext& context) { // Might be better as a separate plugin // All tables of a query are scan tables if the statement both: @@ -202,15 +202,15 @@ proto::ScanInfo ScanTablePlugin::_findScanTables(query::SelectStmt& stmt, query: // Ask css if any of the tables should be locked in memory and their scan rating. // Use this information to determine scanPriority. - proto::ScanInfo scanInfo; + auto scanInfo = protojson::ScanInfo::create(); for (auto& pair : scanTables) { - proto::ScanTableInfo info(pair.first, pair.second); + protojson::ScanTableInfo info(pair.first, pair.second); css::ScanTableParams const params = context.css->getScanTableParams(info.db, info.table); info.lockInMemory = params.lockInMem; info.scanRating = params.scanRating; - scanInfo.infoTables.push_back(info); - scanInfo.scanRating = std::max(scanInfo.scanRating, info.scanRating); - scanInfo.scanRating = std::min(scanInfo.scanRating, static_cast(proto::ScanInfo::SLOWEST)); + scanInfo->infoTables.push_back(info); + scanInfo->scanRating = std::max(scanInfo->scanRating, info.scanRating); + scanInfo->scanRating = std::min(scanInfo->scanRating, static_cast(protojson::ScanInfo::SLOWEST)); LOGS(_log, LOG_LVL_INFO, "ScanInfo " << info.db << "." << info.table << " lockInMemory=" << info.lockInMemory << " rating=" << info.scanRating); diff --git a/src/qana/ScanTablePlugin.h b/src/qana/ScanTablePlugin.h index 145424852f..aa069710c0 100644 --- a/src/qana/ScanTablePlugin.h +++ b/src/qana/ScanTablePlugin.h @@ -27,7 +27,7 @@ #include "qana/QueryPlugin.h" // Qserv headers -#include "proto/ScanTableInfo.h" +#include "protojson/ScanTableInfo.h" namespace lsst::qserv::qana { @@ -55,8 +55,8 @@ class ScanTablePlugin : public QueryPlugin { std::string name() const override { return "ScanTablePlugin"; } private: - proto::ScanInfo _findScanTables(query::SelectStmt& stmt, query::QueryContext& context); - proto::ScanInfo _scanInfo; + protojson::ScanInfo::Ptr _findScanTables(query::SelectStmt& stmt, query::QueryContext& context); + protojson::ScanInfo::Ptr _scanInfo; int _interactiveChunkLimit; }; diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index 74915ec984..f15024d7a2 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -5,13 +5,9 @@ target_sources(qdisp PRIVATE ChunkMeta.cc CzarStats.cc Executive.cc - JobBase.cc JobDescription.cc JobQuery.cc - QdispPool.cc - QueryRequest.cc UberJob.cc - XrdSsiMocks.cc ) target_include_directories(qdisp PRIVATE @@ -47,12 +43,11 @@ target_link_libraries(testQDisp qmeta query rproc - xrdreq Boost::unit_test_framework Threads::Threads ) # This is failing in github actions CI but not when running locally on my dev machine. -# add_test(NAME testQDisp COMMAND testQDisp) +add_test(NAME testQDisp COMMAND testQDisp) # set_tests_properties(testQDisp PROPERTIES WILL_FAIL 1) diff --git a/src/qdisp/CzarStats.cc b/src/qdisp/CzarStats.cc index 3dd53eceb1..c0f19c5014 100644 --- a/src/qdisp/CzarStats.cc +++ b/src/qdisp/CzarStats.cc @@ -29,8 +29,8 @@ // Qserv headers #include "cconfig/CzarConfig.h" -#include "qdisp/QdispPool.h" #include "util/Bug.h" +#include "util/QdispPool.h" #include "util/TimeUtils.h" // LSST headers @@ -46,17 +46,17 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarStats"); namespace lsst::qserv::qdisp { CzarStats::Ptr CzarStats::_globalCzarStats; -util::Mutex CzarStats::_globalMtx; +MUTEX CzarStats::_globalMtx; -void CzarStats::setup(qdisp::QdispPool::Ptr const& qdispPool) { - std::lock_guard lg(_globalMtx); +void CzarStats::setup(util::QdispPool::Ptr const& qdispPool) { + std::lock_guard lg(_globalMtx); if (_globalCzarStats != nullptr || qdispPool == nullptr) { throw util::Bug(ERR_LOC, "Error CzarStats::setup called after global pointer set or qdispPool=null."); } _globalCzarStats = Ptr(new CzarStats(qdispPool)); } -CzarStats::CzarStats(qdisp::QdispPool::Ptr const& qdispPool) +CzarStats::CzarStats(util::QdispPool::Ptr const& qdispPool) : _qdispPool(qdispPool), _startTimeMs(util::TimeUtils::now()) { auto bucketValsRates = {128'000.0, 512'000.0, 1'024'000.0, 16'000'000.0, 128'000'000.0, 256'000'000.0, 512'000'000.0, 768'000'000.0, @@ -77,7 +77,7 @@ CzarStats::CzarStats(qdisp::QdispPool::Ptr const& qdispPool) } CzarStats::Ptr CzarStats::get() { - std::lock_guard lg(_globalMtx); + std::lock_guard lg(_globalMtx); if (_globalCzarStats == nullptr) { throw util::Bug(ERR_LOC, "Error CzarStats::get called before CzarStats::setup."); } diff --git a/src/qdisp/CzarStats.h b/src/qdisp/CzarStats.h index d4dccc8ddf..cddbd8b4e4 100644 --- a/src/qdisp/CzarStats.h +++ b/src/qdisp/CzarStats.h @@ -43,9 +43,11 @@ // Third party headers #include -namespace lsst::qserv::qdisp { - +namespace lsst::qserv::util { class QdispPool; +} + +namespace lsst::qserv::qdisp { /// This class is used to track statistics for the czar. /// setup() needs to be called before get(). @@ -67,7 +69,7 @@ class CzarStats : std::enable_shared_from_this { /// Setup the global CzarStats instance /// @throws Bug if global has already been set or qdispPool is null. - static void setup(std::shared_ptr const& qdispPool); + static void setup(std::shared_ptr const& qdispPool); /// Return a pointer to the global CzarStats instance. /// @throws Bug if get() is called before setup() @@ -147,13 +149,13 @@ class CzarStats : std::enable_shared_from_this { nlohmann::json getTransmitStatsJson() const; private: - CzarStats(std::shared_ptr const& qdispPool); + CzarStats(std::shared_ptr const& qdispPool); - static Ptr _globalCzarStats; ///< Pointer to the global instance. - static util::Mutex _globalMtx; ///< Protects `_globalCzarStats` + static Ptr _globalCzarStats; ///< Pointer to the global instance. + static MUTEX _globalMtx; ///< Protects `_globalCzarStats` /// Connection to get information about the czar's pool of dispatch threads. - std::shared_ptr _qdispPool; + std::shared_ptr _qdispPool; /// The start up time (milliseconds since the UNIX EPOCH) of the status collector. uint64_t const _startTimeMs = 0; diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 0df6f246c7..e88cd488ba 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -48,10 +48,6 @@ // Third-party headers #include "boost/format.hpp" -#include "XrdSsi/XrdSsiErrInfo.hh" -#include "XrdSsi/XrdSsiProvider.hh" -#include "XrdSsi/XrdSsiResource.hh" -#include "XrdSsi/XrdSsiService.hh" // LSST headers #include "lsst/log/Log.h" @@ -62,13 +58,12 @@ #include "ccontrol/msgCode.h" #include "ccontrol/TmpTableName.h" #include "ccontrol/UserQuerySelect.h" +#include "czar/Czar.h" #include "global/LogContext.h" #include "global/ResourceUnit.h" #include "qdisp/CzarStats.h" #include "qdisp/JobQuery.h" -#include "qdisp/QueryRequest.h" #include "qdisp/ResponseHandler.h" -#include "qdisp/XrdSsiMocks.h" #include "query/QueryContext.h" #include "qproc/QuerySession.h" #include "qmeta/Exceptions.h" @@ -80,23 +75,14 @@ #include "util/AsyncTimer.h" #include "util/Bug.h" #include "util/EventThread.h" +#include "util/QdispPool.h" using namespace std; -extern XrdSsiProvider* XrdSsiProviderClient; - namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.Executive"); -string getErrorText(XrdSsiErrInfo& e) { - ostringstream os; - int errCode; - os << "XrdSsiError " << e.Get(errCode); - os << " Code=" << errCode; - return os.str(); -} - } // anonymous namespace namespace lsst::qserv::qdisp { @@ -104,19 +90,18 @@ namespace lsst::qserv::qdisp { //////////////////////////////////////////////////////////////////////// // class Executive implementation //////////////////////////////////////////////////////////////////////// -Executive::Executive(ExecutiveConfig const& c, shared_ptr const& ms, - SharedResources::Ptr const& sharedResources, +Executive::Executive(ExecutiveConfig const& cfg, shared_ptr const& ms, + util::QdispPool::Ptr const& qdispPool, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, shared_ptr const& querySession) - : _config(c), + : _config(cfg), _messageStore(ms), - _qdispPool(sharedResources->getQdispPool()), + _qdispPool(qdispPool), _queryProgress(queryProgress), _queryProgressHistory(queryProgressHistory), _querySession(querySession) { _secondsBetweenQMetaUpdates = chrono::seconds(_config.secondsBetweenChunkUpdates); - _setup(); _setupLimit(); qdisp::CzarStats::get()->addQuery(); } @@ -126,11 +111,10 @@ Executive::~Executive() { qdisp::CzarStats::get()->deleteQuery(); qdisp::CzarStats::get()->deleteJobs(_incompleteJobs.size()); // Remove this executive from the map. - if (czar::Czar::getCzar()->getExecutiveFromMap(getId()) != nullptr) { + auto cz = czar::Czar::getCzar(); // cz can be null in unit tests. + if (cz != nullptr && cz->getExecutiveFromMap(getId()) != nullptr) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) + " pointer in map should be invalid QID=" << getId()); } - // Real XrdSsiService objects are unowned, but mocks are allocated in _setup. - delete dynamic_cast(_xrdSsiService); if (_asyncTimer != nullptr) { _asyncTimer->cancel(); if (_queryProgressHistory != nullptr) { @@ -144,7 +128,7 @@ Executive::~Executive() { } Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptr const& ms, - SharedResources::Ptr const& sharedResources, + std::shared_ptr const& qdispPool, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, shared_ptr const& querySession, @@ -227,13 +211,12 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { // Create the JobQuery and put it in the map. auto jobStatus = make_shared(); Ptr thisPtr = shared_from_this(); - MarkCompleteFunc::Ptr mcf = make_shared(thisPtr, jobDesc->id()); - jobQuery = JobQuery::create(thisPtr, jobDesc, jobStatus, mcf, _id); + jobQuery = JobQuery::create(thisPtr, jobDesc, jobStatus, _id); QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getJobId()); { - lock_guard lock(_cancelled.getMutex()); + lock_guard lock(_cancelled.getMutex()); if (_cancelled) { LOGS(_log, LOG_LVL_DEBUG, "Executive already cancelled, ignoring add(" << jobDesc->id() << ")"); @@ -264,14 +247,20 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { return jobQuery; } -void Executive::runJobQuery(JobQuery::Ptr const& jobQuery) { - bool started = jobQuery->runJob(); - if (!started && isLimitRowComplete()) { - markCompleted(jobQuery->getJobId(), false); +void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 2); + } else { + _qdispPool->queCmd(cmd, 3); } } -void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { +/* &&& +void Executive::queueUberJob(std::shared_ptr const& uberJob) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&&uj queueUberJob"); + auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; + + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(runUberJobFunc)); _jobStartCmdList.push_back(cmd); if (_scanInteractive) { _qdispPool->queCmd(cmd, 0); @@ -279,30 +268,25 @@ void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { _qdispPool->queCmd(cmd, 1); } } +*/ -void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { - if (_scanInteractive) { - _qdispPool->queCmd(cmd, 3); - } else { - _qdispPool->queCmd(cmd, 4); +void Executive::addAndQueueUberJob(shared_ptr const& uj) { + { + lock_guard lck(_uberJobsMapMtx); + UberJobId ujId = uj->getJobId(); + _uberJobsMap[ujId] = uj; + //&&&uj->setAdded(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); } -} -void Executive::runUberJob(std::shared_ptr const& uberJob) { - /// TODO:UJ delete useqdisppool, only set to false if problems during testing - bool const useqdisppool = true; - if (useqdisppool) { - auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; + auto runUberJobFunc = [uj](util::CmdData*) { uj->runUberJob(); }; - auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); - _jobStartCmdList.push_back(cmd); - if (_scanInteractive) { - _qdispPool->queCmd(cmd, 0); - } else { - _qdispPool->queCmd(cmd, 1); - } + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(runUberJobFunc)); + _jobStartCmdList.push_back(cmd); + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 0); } else { - uberJob->runUberJob(); + _qdispPool->queCmd(cmd, 1); } } @@ -319,35 +303,6 @@ void Executive::waitForAllJobsToStart() { LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart done"); } -// If the executive has not been cancelled, then we simply start the query. -// @return true if query was actually started (i.e. we were not cancelled) -// // TODO:UJ delete this function -bool Executive::startQuery(shared_ptr const& jobQuery) { - lock_guard lock(_cancelled.getMutex()); - - // If this has been cancelled, then return false. - if (_cancelled) return false; - - // Construct a temporary resource object to pass to ProcessRequest(). - // Interactive Queries should have an Affinity of XrdSsiResource::None or Weak while - // Scans should have an affinity of Strong - XrdSsiResource::Affinity affinity = (_scanInteractive) ? XrdSsiResource::Weak : XrdSsiResource::Strong; - XrdSsiResource jobResource(jobQuery->getDescription()->resource().path(), "", jobQuery->getIdStr(), "", 0, - affinity); - - // Now construct the actual query request and tie it to the jobQuery. The - // shared pointer is used by QueryRequest to keep itself alive, sloppy design. - // Note that JobQuery calls StartQuery that then calls JobQuery, yech! - // - QueryRequest::Ptr qr = QueryRequest::create(jobQuery); - jobQuery->setQueryRequest(qr); - - // Start the query. The rest is magically done in the background. - // - getXrdSsiService()->ProcessRequest(*(qr.get()), jobResource); - return true; -} - Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { lock_guard lck(_chunkToJobMapMtx); @@ -360,14 +315,6 @@ Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { return unassignedMap; } -void Executive::addUberJobs(std::vector> const& uJobsToAdd) { - lock_guard lck(_uberJobsMapMtx); - for (auto const& uJob : uJobsToAdd) { - UberJobId ujId = uJob->getJobId(); - _uberJobsMap[ujId] = uJob; - } -} - string Executive::dumpUberJobCounts() const { stringstream os; os << "exec=" << getIdStr(); @@ -419,6 +366,7 @@ bool Executive::join() { // To join, we make sure that all of the chunks added so far are complete. // Check to see if _requesters is empty, if not, then sleep on a condition. _waitAllUntilEmpty(); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " wait done"); // Okay to merge. probably not the Executive's responsibility struct successF { static bool func(Executive::JobMap::value_type const& entry) { @@ -437,7 +385,7 @@ bool Executive::join() { if (sCount == _requestCount) { LOGS(_log, LOG_LVL_INFO, "Query execution succeeded all: " << _requestCount << " jobs dispatched and completed."); - } else if (isLimitRowComplete()) { + } else if (isRowLimitComplete()) { LOGS(_log, LOG_LVL_INFO, "Query execution succeeded enough (LIMIT): " << sCount << " jobs out of " << _requestCount << " completed."); @@ -448,15 +396,17 @@ bool Executive::join() { } _empty = (sCount == _requestCount); LOGS(_log, LOG_LVL_DEBUG, - "Flag set to _empty=" << _empty << ", sCount=" << sCount << ", requestCount=" << _requestCount); - return _empty || isLimitRowComplete(); + cName(__func__) << " " + << "Flag set to _empty=" << _empty << ", sCount=" << sCount + << ", requestCount=" << _requestCount); + return _empty || isRowLimitComplete(); } void Executive::markCompleted(JobId jobId, bool success) { ResponseHandler::Error err; string idStr = QueryIdHelper::makeIdStr(_id, jobId); LOGS(_log, LOG_LVL_DEBUG, "Executive::markCompleted " << success); - if (!success && !isLimitRowComplete()) { + if (!success && !isRowLimitComplete()) { { lock_guard lock(_incompleteJobsMutex); auto iter = _incompleteJobs.find(jobId); @@ -496,7 +446,7 @@ void Executive::markCompleted(JobId jobId, bool success) { } } _unTrack(jobId); - if (!success && !isLimitRowComplete()) { + if (!success && !isRowLimitComplete()) { LOGS(_log, LOG_LVL_ERROR, "Executive: requesting squash, cause: " << " failed (code=" << err.getCode() << " " << err.getMsg() << ")"); @@ -524,12 +474,13 @@ void Executive::squash() { job->cancel(); } - // TODO:UJ - Send a message to all workers saying this czarId + queryId is cancelled. - // The workers will just mark all associated tasks as cancelled, and that should be it. - // Any message to this czar about this query should result in an error sent back to - // the worker as soon it can't locate an executive or the executive says cancelled. + // Send a message to all workers saying this czarId + queryId is cancelled. + // The workers will just mark all associated tasks as cancelled, and that should be it. + // Any message to this czar about this query should result in an error sent back to + // the worker as soon it can't locate an executive or the executive says it was + // cancelled. bool const deleteResults = true; - sendWorkerCancelMsg(deleteResults); + sendWorkersEndMsg(deleteResults); LOGS(_log, LOG_LVL_DEBUG, "Executive::squash done"); } @@ -559,18 +510,46 @@ void Executive::_squashSuperfluous() { } bool const keepResults = false; - sendWorkerCancelMsg(keepResults); + sendWorkersEndMsg(keepResults); LOGS(_log, LOG_LVL_DEBUG, "Executive::squashSuperfluous done"); } -void Executive::sendWorkerCancelMsg(bool deleteResults) { - // TODO:UJ need to send a message to the worker that the query is cancelled and all result files - // should be delete - LOGS(_log, LOG_LVL_ERROR, - "TODO:UJ NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId " - "+ " - "queryId. " - << deleteResults); +void Executive::sendWorkersEndMsg(bool deleteResults) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " terminating this query deleteResults=" << deleteResults); + auto cz = czar::Czar::getCzar(); + if (cz != nullptr) { // Possible in unit tests. + cz->getCzarRegistry()->endUserQueryOnWorkers(_id, deleteResults); + } +} + +void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { + if (_cancelled) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " irrelevant as query already cancelled"); + return; + } + + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " killing incomplete UberJobs on " << workerId); + deque ujToCancel; + { + lock_guard lockUJMap(_uberJobsMapMtx); + for (auto const& [ujKey, ujPtr] : _uberJobsMap) { + auto ujStatus = ujPtr->getStatus()->getState(); + if (ujStatus != qmeta::JobStatus::RESPONSE_DONE && ujStatus != qmeta::JobStatus::COMPLETE) { + // RESPONSE_DONE indicates the result file has been read by + // the czar, so before that point the worker's data is + // likely destroyed. COMPLETE indicates all jobs in the + // UberJob are complete. + if (ujPtr->getWorkerContactInfo()->wId == workerId) { + ujToCancel.push_back(ujPtr); + } + } + } + } + + for (auto const& uj : ujToCancel) { + uj->killUberJob(); + uj->setStatusIfOk(qmeta::JobStatus::CANCEL, getIdStr() + " killIncomplete on worker=" + workerId); + } } int Executive::getNumInflight() const { @@ -597,27 +576,6 @@ string Executive::getProgressDesc() const { return msg_progress; } -void Executive::_setup() { - XrdSsiErrInfo eInfo; - _empty.store(true); - _requestCount = 0; - // If unit testing, load the mock service. - if (_config.serviceUrl.compare(_config.getMockStr()) == 0) { - _xrdSsiService = new XrdSsiServiceMock(this); - } else { - static XrdSsiService* xrdSsiServiceStatic = - XrdSsiProviderClient->GetService(eInfo, _config.serviceUrl); - _xrdSsiService = xrdSsiServiceStatic; - } - if (!_xrdSsiService) { - LOGS(_log, LOG_LVL_DEBUG, - _id << " Error obtaining XrdSsiService in Executive: " - "serviceUrl=" - << _config.serviceUrl << " " << getErrorText(eInfo)); - } - assert(_xrdSsiService); -} - /** Add (jobId,r) entry to _requesters map if not here yet * else leave _requesters untouched. * @@ -667,7 +625,7 @@ void Executive::_unTrack(int jobId) { s = _getIncompleteJobsString(5); } } - bool logDebug = untracked || isLimitRowComplete(); + bool logDebug = untracked || isRowLimitComplete(); LOGS(_log, (logDebug ? LOG_LVL_DEBUG : LOG_LVL_WARN), "Executive UNTRACKING " << (untracked ? "success" : "failed") << "::" << s); // Every time a chunk completes, consider sending an update to QMeta. @@ -796,6 +754,13 @@ void Executive::_setupLimit() { _limitSquashApplies = hasLimit && !(groupBy || orderBy || allChunksRequired); } +int Executive::getUjRowLimit() const { + if (_limitSquashApplies) { + return _limit; + } + return 0; +} + void Executive::addResultRows(int64_t rowCount) { _totalResultRows += rowCount; } void Executive::checkLimitRowComplete() { diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index cc68b85a89..4a49955a72 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -39,10 +39,9 @@ #include "global/intTypes.h" #include "global/ResourceUnit.h" #include "global/stringTypes.h" +#include "protojson/ScanTableInfo.h" #include "qdisp/JobDescription.h" #include "qdisp/ResponseHandler.h" -#include "qdisp/SharedResources.h" -#include "qdisp/QdispPool.h" #include "qdisp/UberJob.h" #include "qmeta/JobStatus.h" #include "util/EventThread.h" @@ -51,8 +50,6 @@ #include "util/threadSafe.h" #include "util/ThreadPool.h" -// TODO:UJ replace with better enable/disable feature, or just use only UberJobs -#define uberJobsEnabled 1 // Forward declarations class XrdSsiService; @@ -62,6 +59,9 @@ class QProgress; class QProgressHistory; } // namespace lsst::qserv::qmeta + +namespace lsst::qserv { + namespace ccontrol { class UserQuerySelect; } @@ -70,11 +70,11 @@ namespace qmeta { class MessageStore; } // namespace qmeta -namespace lsst::qserv::qproc { +namespace qproc { class QuerySession; } // namespace lsst::qserv::qproc -namespace lsst::qserv::qdisp { +namespace qdisp { class JobQuery; class UberJob; } // namespace qdisp @@ -82,7 +82,9 @@ class UberJob; namespace util { class AsyncTimer; -} // namespace lsst::qserv::util +class PriorityCommand; +class QdispPool; +} // namespace util // This header declarations namespace lsst::qserv::qdisp { @@ -110,15 +112,17 @@ class Executive : public std::enable_shared_from_this { /// If c->serviceUrl == ExecutiveConfig::getMockStr(), then use XrdSsiServiceMock /// instead of a real XrdSsiService static Executive::Ptr create(ExecutiveConfig const& c, std::shared_ptr const& ms, - SharedResources::Ptr const& sharedResources, + std::shared_ptr const& qdispPool, std::shared_ptr const& queryProgress, std::shared_ptr const& queryProgressHistory, std::shared_ptr const& querySession, boost::asio::io_service& asioIoService); - ~Executive(); + virtual ~Executive(); - std::string cName(const char* funcName = "") { return std::string("Executive::") + funcName; } + std::string cName(const char* funcName = "") { + return std::string("Executive::") + funcName + " " + getIdStr(); + } /// Set the UserQuerySelect object for this query so this Executive can ask it to make new /// UberJobs in the future, if needed. @@ -133,17 +137,11 @@ class Executive : public std::enable_shared_from_this { /// Add an item with a reference number std::shared_ptr add(JobDescription::Ptr const& s); - /// TODO:UJ - to be deleted - void runJobQuery(std::shared_ptr const& jobQuery); - - // Queue `uberJob` to be run using the QDispPool. - void runUberJob(std::shared_ptr const& uberJob); - - /// Queue a job to be sent to a worker so it can be started. - void queueJobStart(PriorityCommand::Ptr const& cmd); + /// Add the UberJob `uj` to the list and queue it to be sent to a worker. + void addAndQueueUberJob(std::shared_ptr const& uj); /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. - void queueFileCollect(PriorityCommand::Ptr const& cmd); + void queueFileCollect(std::shared_ptr const& cmd); /// Waits for all jobs on _jobStartCmdList to start. This should not be called /// before ALL jobs have been added to the pool. @@ -170,6 +168,7 @@ class Executive : public std::enable_shared_from_this { std::string const& getIdStr() const { return _idStr; } void setScanInteractive(bool interactive) { _scanInteractive = interactive; } + bool getScanInteractive() const { return _scanInteractive; } /// @return number of jobs in flight. int getNumInflight() const; @@ -180,11 +179,7 @@ class Executive : public std::enable_shared_from_this { /// @return true if cancelled bool getCancelled() { return _cancelled; } - XrdSsiService* getXrdSsiService() { return _xrdSsiService; } - - std::shared_ptr getQdispPool() { return _qdispPool; } - - bool startQuery(std::shared_ptr const& jobQuery); // TODO:UJ delete + std::shared_ptr getQdispPool() { return _qdispPool; } /// Add 'rowCount' to the total number of rows in the result table. void addResultRows(int64_t rowCount); @@ -195,9 +190,13 @@ class Executive : public std::enable_shared_from_this { /// rows already read in. void checkLimitRowComplete(); - /// @return _limitRowComplete, which can only be meaningful if the + /// Returns the maximum number of rows the worker needs for the LIMIT clause, or + /// a value <= 0 there's no limit that can be applied at the worker. + int getUjRowLimit() const; + + /// @return _rowLimitComplete, which can only be meaningful if the /// user query has not been cancelled. - bool isLimitRowComplete() { return _limitRowComplete && !_cancelled; } + bool isRowLimitComplete() { return _rowLimitComplete && !_cancelled; } /// @return the value of _dataIgnoredCount int incrDataIgnoredCount() { return ++_dataIgnoredCount; } @@ -206,22 +205,12 @@ class Executive : public std::enable_shared_from_this { /// @see python module lsst.qserv.czar.proxy.unlock() void updateProxyMessages(); - /// Add UbjerJobs to this user query. - void addUberJobs(std::vector> const& jobsToAdd); - /// Call UserQuerySelect::buildAndSendUberJobs make new UberJobs for /// unassigned jobs. - void assignJobsToUberJobs(); + virtual void assignJobsToUberJobs(); int getTotalJobs() { return _totalJobs; } - /// Set `_failedUberJob` to `val`; Setting this to true is a flag - /// that indicates to the Czar::_monitor that this Executive - /// probably has unassigned jobs that need to be placed in - /// new UberJobs. This `val` should only be set false by - /// Czar::_monitor(). - void setFlagFailedUberJob(bool val) { _failedUberJob = val; } - /// Add an error code and message that may be displayed to the user. void addMultiError(int errorCode, std::string const& errorMsg, int errState); @@ -239,16 +228,26 @@ class Executive : public std::enable_shared_from_this { /// Send a message to all workers to cancel this query. /// @param deleteResults - If true, delete all result files for this query on the workers. - void sendWorkerCancelMsg(bool deleteResults); + void sendWorkersEndMsg(bool deleteResults); -private: - Executive(ExecutiveConfig const& c, std::shared_ptr const& ms, - SharedResources::Ptr const& sharedResources, + /// Complete UberJobs have their results on the czar, the + /// incomplete UberJobs need to be stopped and possibly reassigned. + void killIncompleteUberJobsOnWorker(std::string const& workerId); + + // Try to remove this and put in constructor + void setScanInfo(protojson::ScanInfo::Ptr const& scanInfo) { _scanInfo = scanInfo; } + + /// Return a pointer to _scanInfo. + protojson::ScanInfo::Ptr getScanInfo() { return _scanInfo; } + +protected: + Executive(ExecutiveConfig const& cfg, std::shared_ptr const& ms, + std::shared_ptr const& qdispPool, std::shared_ptr const& queryProgress, std::shared_ptr const& queryProgressHistory, std::shared_ptr const& querySession); - void _setup(); +private: void _setupLimit(); bool _track(int refNum, std::shared_ptr const& r); @@ -260,10 +259,10 @@ class Executive : public std::enable_shared_from_this { void _squashSuperfluous(); - /// @return previous value of _limitRowComplete while setting it to true. + /// @return previous value of _rowLimitComplete while setting it to true. /// This indicates that enough rows have been read to complete the user query /// with a LIMIT clause, and no group by or order by clause. - bool _setLimitRowComplete() { return _limitRowComplete.exchange(true); } + bool _setLimitRowComplete() { return _rowLimitComplete.exchange(true); } // for debugging void _printState(std::ostream& os); @@ -276,22 +275,20 @@ class Executive : public std::enable_shared_from_this { std::atomic _empty{true}; std::shared_ptr _messageStore; ///< MessageStore for logging - /// RPC interface, static to avoid getting every time a user query starts and separate - /// from _xrdSsiService to avoid conflicts with XrdSsiServiceMock. - XrdSsiService* _xrdSsiService; ///< RPC interface - JobMap _jobMap; ///< Contains information about all jobs. - JobMap _incompleteJobs; ///< Map of incomplete jobs. + JobMap _jobMap; ///< Contains information about all jobs. + JobMap _incompleteJobs; ///< Map of incomplete jobs. /// How many jobs are used in this query. 1 avoids possible 0 of 0 jobs completed race condition. /// The correct value is set when it is available. std::atomic _totalJobs{1}; - QdispPool::Ptr _qdispPool; ///< Shared thread pool for handling commands to and from workers. + std::shared_ptr + _qdispPool; ///< Shared thread pool for handling commands to and from workers. - std::deque _jobStartCmdList; ///< list of jobs to start. + std::deque> _jobStartCmdList; ///< list of jobs to start. /** Execution errors */ util::MultiError _multiError; - std::atomic _requestCount; ///< Count of submitted jobs + std::atomic _requestCount{0}; ///< Count of submitted jobs util::Flag _cancelled{false}; ///< Has execution been cancelled. // Mutexes @@ -318,7 +315,8 @@ class Executive : public std::enable_shared_from_this { std::chrono::seconds _secondsBetweenQMetaUpdates{60}; std::mutex _lastQMetaMtx; ///< protects _lastQMetaUpdate. - bool _scanInteractive = false; ///< true for interactive scans. + /// true for interactive scans, once set it doesn't change. + bool _scanInteractive = false; // Add a job to the _chunkToJobMap // TODO:UJ This may need review as large changes were made to this part of the code. @@ -334,7 +332,7 @@ class Executive : public std::enable_shared_from_this { /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. - std::atomic _limitRowComplete{false}; + std::atomic _rowLimitComplete{false}; std::atomic _totalResultRows{0}; std::weak_ptr _querySession; @@ -352,32 +350,10 @@ class Executive : public std::enable_shared_from_this { /// Weak pointer to the UserQuerySelect object for this query. std::weak_ptr _userQuerySelect; - /// If this is true, there are probably jobs that need to - /// be reassigned to new UberJobs. - std::atomic _failedUberJob{false}; - /// Flag that is set to true when ready to create and run UberJobs. std::atomic _readyToExecute{false}; -}; -/// TODO:UJ delete - MarkCompleteFunc is not needed with uberjobs. -class MarkCompleteFunc { -public: - typedef std::shared_ptr Ptr; - - MarkCompleteFunc(Executive::Ptr const& e, JobId jobId) : _executive(e), _jobId(jobId) {} - virtual ~MarkCompleteFunc() {} - - virtual void operator()(bool success) { - auto exec = _executive.lock(); - if (exec != nullptr) { - exec->markCompleted(_jobId, success); - } - } - -private: - std::weak_ptr _executive; - JobId _jobId; + protojson::ScanInfo::Ptr _scanInfo; ///< Scan rating and tables. }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobBase.cc b/src/qdisp/JobBase.cc deleted file mode 100644 index a5ef5a8c8f..0000000000 --- a/src/qdisp/JobBase.cc +++ /dev/null @@ -1,54 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "qdisp/JobBase.h" - -// System headers -#include - -// Qserv headers - -// LSST headers -#include "lsst/log/Log.h" - -using namespace std; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobBase"); -} - -namespace lsst { namespace qserv { namespace qdisp { - -std::ostream& JobBase::dumpOS(std::ostream& os) const { - os << "JobBase no data members"; - return os; -} - -std::string JobBase::dump() const { - std::ostringstream os; - dumpOS(os); - return os.str(); -} - -std::ostream& operator<<(std::ostream& os, JobBase const& jb) { return jb.dumpOS(os); } - -}}} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h deleted file mode 100644 index e5df5fc2ab..0000000000 --- a/src/qdisp/JobBase.h +++ /dev/null @@ -1,78 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_QDISP_JOBBASE_H -#define LSST_QSERV_QDISP_JOBBASE_H - -// System headers -#include -#include - -// Qserv headers -#include "global/intTypes.h" - -namespace lsst::qserv::qmeta { -class JobStatus; -} - -// This header declarations -namespace lsst::qserv::qdisp { - -class Executive; -class QdispPool; -class ResponseHandler; -class QueryRequest; - -/// Base class for JobQuery and UberJob. -/// TODO:UJ This could use a lot of cleanup. Once UberJobs are fully in effect, there's no need -/// for this base class as it won't be possible to send a JobQuery to a worker without -/// putting it in an UberJob first. The UberJob is a wrapper that stores worker contact -/// info. -class JobBase : public std::enable_shared_from_this { -public: - using Ptr = std::shared_ptr; - - JobBase() = default; - JobBase(JobBase const&) = delete; - JobBase& operator=(JobBase const&) = delete; - virtual ~JobBase() = default; - - virtual QueryId getQueryId() const = 0; - virtual UberJobId getJobId() const = 0; - virtual std::string const& getIdStr() const = 0; - virtual std::shared_ptr getQdispPool() = 0; - virtual std::string const& getPayload() const = 0; ///< const& in return type is essential for xrootd - virtual std::shared_ptr getRespHandler() = 0; - virtual std::shared_ptr getStatus() = 0; - virtual bool getScanInteractive() const = 0; - virtual bool isQueryCancelled() = 0; - virtual void callMarkCompleteFunc(bool success) = 0; - virtual void setQueryRequest(std::shared_ptr const& qr) = 0; - virtual std::shared_ptr getExecutive() = 0; - - virtual std::ostream& dumpOS(std::ostream& os) const; - - std::string dump() const; - friend std::ostream& operator<<(std::ostream& os, JobBase const& jb); -}; - -} // namespace lsst::qserv::qdisp - -#endif // LSST_QSERV_QDISP_JOBBASE_H diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index c8ade0f58e..660e573305 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -33,13 +33,11 @@ #include "lsst/log/Log.h" // Qserv headers -#include "proto/ProtoImporter.h" #include "proto/worker.pb.h" #include "util/Bug.h" #include "qdisp/Executive.h" #include "qdisp/ResponseHandler.h" #include "qproc/ChunkQuerySpec.h" -#include "qproc/TaskMsgFactory.h" using namespace std; @@ -51,7 +49,6 @@ namespace lsst::qserv::qdisp { JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, shared_ptr const& respHandler, - shared_ptr const& taskMsgFactory, shared_ptr const& chunkQuerySpec, string const& chunkResultName, bool mock) : _czarId(czarId), @@ -60,22 +57,11 @@ JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, R _qIdStr(QueryIdHelper::makeIdStr(_queryId, _jobId)), _resource(resource), _respHandler(respHandler), - _taskMsgFactory(taskMsgFactory), _chunkQuerySpec(chunkQuerySpec), _chunkResultName(chunkResultName), _mock(mock) {} -bool JobDescription::incrAttemptCount() { - ++_attemptCount; - if (_attemptCount > MAX_JOB_ATTEMPTS) { - LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); - return false; - } - buildPayload(); - return true; -} - -bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase) { +bool JobDescription::incrAttemptCount(std::shared_ptr const& exec, bool increase) { if (increase) { ++_attemptCount; } @@ -98,39 +84,16 @@ bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr return false; } } - - // build the request - auto js = _taskMsgFactory->makeMsgJson(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, - _attemptCount, _czarId); - LOGS(_log, LOG_LVL_DEBUG, "JobDescription::" << __func__ << " js=" << (*js)); - _jsForWorker = js; - - return true; -} - -void JobDescription::buildPayload() { - ostringstream os; - _taskMsgFactory->serializeMsg(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, _attemptCount, - _czarId, os); - _payloads[_attemptCount] = os.str(); -} - -bool JobDescription::verifyPayload() const { // TODO:UJ delete - proto::ProtoImporter pi; - if (!_mock && !pi.messageAcceptable(_payloads.at(_attemptCount))) { - LOGS(_log, LOG_LVL_DEBUG, _qIdStr << " Error serializing TaskMsg."); - return false; - } return true; } bool JobDescription::getScanInteractive() const { return _chunkQuerySpec->scanInteractive; } -int JobDescription::getScanRating() const { return _chunkQuerySpec->scanInfo.scanRating; } +int JobDescription::getScanRating() const { return _chunkQuerySpec->scanInfo->scanRating; } ostream& operator<<(ostream& os, JobDescription const& jd) { - os << "job(id=" << jd._jobId << " payloads.size=" << jd._payloads.size() << " ru=" << jd._resource.path() - << " attemptCount=" << jd._attemptCount << ")"; + os << "job(id=" << jd._jobId << " ru=" << jd._resource.path() << " attemptCount=" << jd._attemptCount + << ")"; return os; } diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index bc69edb25a..75ca4a33b3 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -29,6 +29,7 @@ // System headers #include +#include #include // Third party headers @@ -44,13 +45,8 @@ namespace lsst::qserv { -namespace proto { -class TaskMsg; -} - namespace qproc { class ChunkQuerySpec; -class TaskMsgFactory; } // namespace qproc namespace qdisp { @@ -58,18 +54,16 @@ namespace qdisp { class Executive; class ResponseHandler; -/** Description of a job managed by the executive - */ +/// Description of a job managed by the executive class JobDescription { public: using Ptr = std::shared_ptr; static JobDescription::Ptr create(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, std::shared_ptr const& respHandler, - std::shared_ptr const& taskMsgFactory, std::shared_ptr const& chunkQuerySpec, std::string const& chunkResultName, bool mock = false) { - JobDescription::Ptr jd(new JobDescription(czarId, qId, jobId, resource, respHandler, taskMsgFactory, + JobDescription::Ptr jd(new JobDescription(czarId, qId, jobId, resource, respHandler, chunkQuerySpec, chunkResultName, mock)); return jd; } @@ -77,30 +71,28 @@ class JobDescription { JobDescription(JobDescription const&) = delete; JobDescription& operator=(JobDescription const&) = delete; - void buildPayload(); ///< Must be run after construction to avoid problems with unit tests. JobId id() const { return _jobId; } ResourceUnit const& resource() const { return _resource; } - std::string const& payload() { return _payloads[_attemptCount]; } std::shared_ptr respHandler() { return _respHandler; } int getAttemptCount() const { return _attemptCount; } + std::shared_ptr getChunkQuerySpec() { return _chunkQuerySpec; } + std::string getChunkResultName() { return _chunkResultName; } bool getScanInteractive() const; int getScanRating() const; - /// @returns true when _attemptCount is incremented correctly and the payload is built. - bool incrAttemptCount(); - bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. + /// Increase the attempt count by 1 and return false if that puts it over the limit. + bool incrAttemptCount(std::shared_ptr const& exec, bool increase); std::shared_ptr getJsForWorker() { return _jsForWorker; } - void resetJsForWorker() { _jsForWorker.reset(); } // TODO:UJ may need mutex for _jsForWorker + void resetJsForWorker() { _jsForWorker.reset(); } friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); private: JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, std::shared_ptr const& respHandler, - std::shared_ptr const& taskMsgFactory, std::shared_ptr const& chunkQuerySpec, std::string const& chunkResultName, bool mock = false); @@ -111,14 +103,7 @@ class JobDescription { int _attemptCount{-1}; ///< Start at -1 so that first attempt will be 0, see incrAttemptCount(). ResourceUnit _resource; ///< path, e.g. /q/LSST/23125 - /// _payloads - encoded requests, one per attempt. No guarantee that xrootd is done - /// with the payload buffer, so hang onto all of them until the query is finished. - /// Also, using a map so the strings wont be moved. - /// The xrootd callback function QueryRequest::GetRequest should - /// return something other than a char*. - std::map _payloads; std::shared_ptr _respHandler; // probably MergingHandler - std::shared_ptr _taskMsgFactory; std::shared_ptr _chunkQuerySpec; std::string _chunkResultName; diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 9149311968..71d9f19ecc 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -34,7 +34,6 @@ // Qserv headers #include "global/LogContext.h" #include "qdisp/Executive.h" -#include "qdisp/QueryRequest.h" namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobQuery"); @@ -45,16 +44,12 @@ using namespace std; namespace lsst::qserv::qdisp { JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - qmeta::JobStatus::Ptr const& jobStatus, - shared_ptr const& markCompleteFunc, QueryId qid) - : JobBase(), - _executive(executive), + qmeta::JobStatus::Ptr const& jobStatus, QueryId qid) + : _executive(executive), _jobDescription(jobDescription), - _markCompleteFunc(markCompleteFunc), _jobStatus(jobStatus), _qid(qid), _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { - _qdispPool = executive->getQdispPool(); LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } @@ -63,96 +58,26 @@ JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_WARN, "~JobQuery QID=" << _idStr); } -/** Attempt to run the job on a worker. - * @return - false if it can not setup the job or the maximum number of attempts has been reached. - */ -bool JobQuery::runJob() { // TODO:UJ delete - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); - LOGS(_log, LOG_LVL_DEBUG, " runJob " << *this); - auto executive = _executive.lock(); - if (executive == nullptr) { - LOGS(_log, LOG_LVL_ERROR, "runJob failed executive==nullptr"); - - return false; - } - bool superfluous = executive->isLimitRowComplete(); - bool cancelled = executive->getCancelled(); - bool handlerReset = _jobDescription->respHandler()->reset(); - if (!(cancelled || superfluous) && handlerReset) { - auto criticalErr = [this, &executive](string const& msg) { - LOGS(_log, LOG_LVL_ERROR, msg << " " << _jobDescription << " Canceling user query!"); - executive->squash(); // This should kill all jobs in this user query. - }; - - LOGS(_log, LOG_LVL_DEBUG, "runJob checking attempt=" << _jobDescription->getAttemptCount()); - lock_guard lock(_rmutex); - if (_jobDescription->getAttemptCount() < _getMaxAttempts()) { - bool okCount = _jobDescription->incrAttemptCount(); - if (!okCount) { - criticalErr("hit structural max of retries"); - return false; - } - if (!_jobDescription->verifyPayload()) { - criticalErr("bad payload"); - return false; - } - } else { - LOGS(_log, LOG_LVL_DEBUG, "runJob max retries"); - criticalErr("hit maximum number of retries"); - return false; - } - - // At this point we are all set to actually run the query. We create a - // a shared pointer to this object to prevent it from escaping while we - // are trying to start this whole process. We also make sure we record - // whether or not we are in SSI as cancellation handling differs. - // - LOGS(_log, LOG_LVL_TRACE, "runJob calls StartQuery()"); - JobQuery::Ptr jq(dynamic_pointer_cast(shared_from_this())); - _inSsi = true; - if (executive->startQuery(jq)) { - _jobStatus->updateInfo(_idStr, qmeta::JobStatus::REQUEST, "EXEC"); - return true; - } - _inSsi = false; - } - LOGS(_log, (superfluous ? LOG_LVL_DEBUG : LOG_LVL_WARN), - "runJob failed. cancelled=" << cancelled << " reset=" << handlerReset); - return false; -} - /// Cancel response handling. Return true if this is the first time cancel has been called. bool JobQuery::cancel(bool superfluous) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_DEBUG, "JobQuery::cancel()"); if (_cancelled.exchange(true) == false) { - lock_guard lock(_rmutex); - // If _inSsi is true then this query request has been passed to SSI and - // _queryRequestPtr cannot be a nullptr. Cancellation is complicated. - bool cancelled = false; - if (_inSsi) { - LOGS(_log, LOG_LVL_DEBUG, "cancel QueryRequest in progress"); - if (_queryRequestPtr->cancel()) { - LOGS(_log, LOG_LVL_DEBUG, "cancelled by QueryRequest"); - cancelled = true; - } else { - LOGS(_log, LOG_LVL_DEBUG, "QueryRequest could not cancel"); - } + VMUTEX_NOT_HELD(_jqMtx); + lock_guard lock(_jqMtx); + + ostringstream os; + os << _idStr << " cancel"; + LOGS(_log, LOG_LVL_DEBUG, os.str()); + if (!superfluous) { + getDescription()->respHandler()->errorFlush(os.str(), -1); } - if (!cancelled) { - ostringstream os; - os << _idStr << " cancel QueryRequest=" << _queryRequestPtr; - LOGS(_log, LOG_LVL_DEBUG, os.str()); - if (!superfluous) { - getDescription()->respHandler()->errorFlush(os.str(), -1); - } - auto executive = _executive.lock(); - if (executive == nullptr) { - LOGS(_log, LOG_LVL_ERROR, " can't markComplete cancelled, executive == nullptr"); - return false; - } - executive->markCompleted(getJobId(), false); + auto executive = _executive.lock(); + if (executive == nullptr) { + LOGS(_log, LOG_LVL_ERROR, " can't markComplete cancelled, executive == nullptr"); + return false; } + executive->markCompleted(getJobId(), false); if (!superfluous) { _jobDescription->respHandler()->processCancel(); } @@ -178,6 +103,7 @@ bool JobQuery::isQueryCancelled() { bool JobQuery::_setUberJobId(UberJobId ujId) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); + VMUTEX_HELD(_jqMtx); if (_uberJobId >= 0 && ujId != _uberJobId) { LOGS(_log, LOG_LVL_DEBUG, __func__ << " couldn't change UberJobId as ujId=" << ujId << " is owned by " << _uberJobId); @@ -189,7 +115,8 @@ bool JobQuery::_setUberJobId(UberJobId ujId) { bool JobQuery::unassignFromUberJob(UberJobId ujId) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); - std::lock_guard lock(_rmutex); + VMUTEX_NOT_HELD(_jqMtx); + lock_guard lock(_jqMtx); if (_uberJobId < 0) { LOGS(_log, LOG_LVL_INFO, __func__ << " UberJobId already unassigned. attempt by ujId=" << ujId); return true; @@ -203,21 +130,26 @@ bool JobQuery::unassignFromUberJob(UberJobId ujId) { auto exec = _executive.lock(); // Do not increase the count as it should have been increased when the job was started. - _jobDescription->incrAttemptCountScrubResultsJson(exec, false); + _jobDescription->incrAttemptCount(exec, false); return true; } int JobQuery::getAttemptCount() const { - std::lock_guard lock(_rmutex); + VMUTEX_NOT_HELD(_jqMtx); + lock_guard lock(_jqMtx); return _jobDescription->getAttemptCount(); } -string const& JobQuery::getPayload() const { return _jobDescription->payload(); } - -void JobQuery::callMarkCompleteFunc(bool success) { _markCompleteFunc->operator()(success); } - ostream& JobQuery::dumpOS(ostream& os) const { return os << "{" << getIdStr() << _jobDescription << " " << _jobStatus << "}"; } +std::string JobQuery::dump() const { + std::ostringstream os; + dumpOS(os); + return os.str(); +} + +std::ostream& operator<<(std::ostream& os, JobQuery const& jq) { return jq.dumpOS(os); } + } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index a11b628d49..7c22d7f74d 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -33,87 +33,58 @@ // Qserv headers #include "qdisp/Executive.h" -#include "qdisp/JobBase.h" #include "qdisp/JobDescription.h" #include "qdisp/ResponseHandler.h" #include "util/InstanceCount.h" +#include "util/Mutex.h" namespace lsst::qserv::qdisp { -class QdispPool; class QueryRequest; -/// This class is used to describe, monitor, and control a single query to a worker. -/// TODO:UJ once all Jobs are sent out as UberJobs, the purpose of this class is a bit -/// vague. It's components should probably be split between UberJob and -/// JobDescription. -class JobQuery : public JobBase { +/// This class is used to describe and monitor the queries for a +/// chunk on the worker. +class JobQuery { public: typedef std::shared_ptr Ptr; /// Factory function to make certain a shared_ptr is used and _setup is called. static JobQuery::Ptr create(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - qmeta::JobStatus::Ptr const& jobStatus, - std::shared_ptr const& markCompleteFunc, QueryId qid) { - Ptr jq = Ptr(new JobQuery(executive, jobDescription, jobStatus, markCompleteFunc, qid)); - jq->_setup(); + qmeta::JobStatus::Ptr const& jobStatus, QueryId qid) { + Ptr jq = Ptr(new JobQuery(executive, jobDescription, jobStatus, qid)); return jq; } virtual ~JobQuery(); - /// Run this job. - bool runJob(); - - QueryId getQueryId() const override { return _qid; } - JobId getJobId() const override { return _jobDescription->id(); } - std::string const& getPayload() const override; - std::string const& getIdStr() const override { return _idStr; } - std::shared_ptr getRespHandler() override { return _jobDescription->respHandler(); } - bool getScanInteractive() const override { return _jobDescription->getScanInteractive(); } + QueryId getQueryId() const { return _qid; } + JobId getJobId() const { return _jobDescription->id(); } + std::string const& getIdStr() const { return _idStr; } + std::shared_ptr getRespHandler() { return _jobDescription->respHandler(); } JobDescription::Ptr getDescription() { return _jobDescription; } - - qmeta::JobStatus::Ptr getStatus() override { return _jobStatus; } - - void setQueryRequest(std::shared_ptr const& qr) { - std::lock_guard lock(_rmutex); - _queryRequestPtr = qr; - } - std::shared_ptr getQueryRequest() { - std::lock_guard lock(_rmutex); - return _queryRequestPtr; - } - - void callMarkCompleteFunc(bool success) override; + qmeta::JobStatus::Ptr getStatus() { return _jobStatus; } bool cancel(bool superfluous = false); - bool isQueryCancelled() override; + bool isQueryCancelled(); - std::shared_ptr getExecutive() override { return _executive.lock(); } - - std::shared_ptr getQdispPool() override { return _qdispPool; } - - std::ostream& dumpOS(std::ostream& os) const override; - - /// Make a copy of the job description. JobQuery::_setup() must be called after creation. - /// Do not call this directly, use create. - JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - qmeta::JobStatus::Ptr const& jobStatus, - std::shared_ptr const& markCompleteFunc, QueryId qid); + std::shared_ptr getExecutive() { return _executive.lock(); } /// If the UberJob is unassigned, change the _uberJobId to ujId. bool setUberJobId(UberJobId ujId) { - std::lock_guard lock(_rmutex); + VMUTEX_NOT_HELD(_jqMtx); + std::lock_guard lock(_jqMtx); return _setUberJobId(ujId); } UberJobId getUberJobId() const { - std::lock_guard lock(_rmutex); + VMUTEX_NOT_HELD(_jqMtx); + std::lock_guard lock(_jqMtx); return _getUberJobId(); } bool isInUberJob() const { - std::lock_guard lock(_rmutex); + VMUTEX_NOT_HELD(_jqMtx); + std::lock_guard lock(_jqMtx); return _isInUberJob(); } @@ -123,28 +94,37 @@ class JobQuery : public JobBase { /// @return true if job is unassigned. bool unassignFromUberJob(UberJobId ujId); + std::ostream& dumpOS(std::ostream& os) const; + std::string dump() const; + friend std::ostream& operator<<(std::ostream& os, JobQuery const& jq); + protected: - void _setup() { - JobBase::Ptr jbPtr = shared_from_this(); - _jobDescription->respHandler()->setJobQuery(jbPtr); - } + /// Make a copy of the job description. JobQuery::_setup() must be called after creation. + /// Do not call this directly, use create. + JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, + qmeta::JobStatus::Ptr const& jobStatus, QueryId qid); /// @return true if _uberJobId was set, it can only be set if it is unassigned /// or by the current owner. - /// NOTE: _rmutex must be held before calling this + /// NOTE: _jqMtx must be held before calling this bool _setUberJobId(UberJobId ujId); - /// NOTE: _rmutex must be held before calling this - UberJobId _getUberJobId() const { return _uberJobId; } + /// NOTE: _jqMtx must be held before calling this + UberJobId _getUberJobId() const { + VMUTEX_HELD(_jqMtx); + return _uberJobId; + } - /// NOTE: _rmutex must be held before calling this - bool _isInUberJob() const { return _uberJobId >= 0; } + /// NOTE: _jqMtx must be held before calling this + bool _isInUberJob() const { + VMUTEX_HELD(_jqMtx); + return _uberJobId >= 0; + } // Values that don't change once set. std::weak_ptr _executive; /// The job description needs to survive until the task is complete. JobDescription::Ptr _jobDescription; - std::shared_ptr _markCompleteFunc; // JobStatus has its own mutex. qmeta::JobStatus::Ptr _jobStatus; ///< Points at status in Executive::_statusMap @@ -153,20 +133,11 @@ class JobQuery : public JobBase { std::string const _idStr; ///< Identifier string for logging. // Values that need mutex protection - // TODO:UJ recursive can probably go away with as well as _inSsi. - mutable std::recursive_mutex _rmutex; ///< protects _jobDescription, - ///< _queryRequestPtr, _uberJobId, - ///< and _inSsi - - // SSI items - std::shared_ptr _queryRequestPtr; - bool _inSsi{false}; + mutable MUTEX _jqMtx; ///< protects _jobDescription, _queryRequestPtr, _uberJobId // Cancellation std::atomic _cancelled{false}; ///< Lock to make sure cancel() is only called once. - std::shared_ptr _qdispPool; - /// The UberJobId that this job is assigned to. Values less than zero /// indicate this job is unassigned. To prevent race conditions, /// an UberJob may only unassign a job if it has the same ID as diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index c6912f415b..dab0942b98 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -42,7 +42,8 @@ class ResponseSummary; namespace lsst::qserv::qdisp { -class JobBase; +class JobQuery; +class UberJob; /// ResponseHandler is an interface that handles result bytes. Tasks are /// submitted to an Executive instance naming a resource unit (what resource is @@ -57,7 +58,7 @@ class ResponseHandler { typedef std::shared_ptr Ptr; ResponseHandler() {} - void setJobQuery(std::shared_ptr const& jobBase) { _jobBase = jobBase; } + void setUberJob(std::weak_ptr const& ujPtr) { _uberJob = ujPtr; } virtual ~ResponseHandler() {} /// Process a request for pulling and merging a job result into the result table @@ -79,10 +80,6 @@ class ResponseHandler { /// Signal an unrecoverable error condition. No further calls are expected. virtual void errorFlush(std::string const& msg, int code) = 0; - /// @return true if the receiver has completed its duties. - virtual bool finished() const = 0; - virtual bool reset() = 0; ///< Reset the state that a request can be retried. - /// Print a string representation of the receiver to an ostream virtual std::ostream& print(std::ostream& os) const = 0; @@ -92,10 +89,13 @@ class ResponseHandler { /// Do anything that needs to be done if this job gets cancelled. virtual void processCancel() {}; - std::weak_ptr getJobQuery() { return _jobQuery; } + /// Scrub the results from jobId-attempt from the result table. + virtual void prepScrubResults(int jobId, int attempt) = 0; + + std::weak_ptr getUberJob() { return _uberJob; } private: - std::weak_ptr _jobBase; + std::weak_ptr _uberJob; }; inline std::ostream& operator<<(std::ostream& os, ResponseHandler const& r) { return r.print(os); } diff --git a/src/qdisp/SharedResources.h b/src/qdisp/SharedResources.h deleted file mode 100644 index 37d06f701e..0000000000 --- a/src/qdisp/SharedResources.h +++ /dev/null @@ -1,60 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_SHAREDRESOURCES_H -#define LSST_QSERV_SHAREDRESOURCES_H - -// System headers -#include - -namespace lsst::qserv::qdisp { - -class QdispPool; - -/// Put resources that all Executives need to share in one class to reduce -/// the number of arguments passed. -/// This class should be kept simple so it can easily be included in headers -/// without undue compiler performances problems. -class SharedResources { -public: - using Ptr = std::shared_ptr; - - static Ptr create(std::shared_ptr const& qdispPool) { - return Ptr(new SharedResources(qdispPool)); - } - - SharedResources() = delete; - SharedResources(SharedResources const&) = delete; - SharedResources& operator=(SharedResources const&) = delete; - ~SharedResources() = default; - - std::shared_ptr getQdispPool() { return _qdispPool; } - -private: - SharedResources(std::shared_ptr const& qdispPool) : _qdispPool(qdispPool) {} - - /// Thread pool for handling Responses from XrdSsi. - std::shared_ptr _qdispPool; -}; - -} // namespace lsst::qserv::qdisp - -#endif // LSST_QSERV_SHAREDRESOURCES_H diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 8b092a6d3b..10f535ff13 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -31,16 +31,20 @@ #include "nlohmann/json.hpp" // Qserv headers +#include "czar/Czar.h" #include "cconfig/CzarConfig.h" #include "global/LogContext.h" #include "http/Client.h" #include "http/MetaModule.h" -#include "proto/ProtoImporter.h" #include "proto/worker.pb.h" +#include "protojson/UberJobMsg.h" #include "qdisp/JobQuery.h" #include "qmeta/JobStatus.h" +#include "qproc/ChunkQuerySpec.h" #include "util/Bug.h" #include "util/common.h" +#include "util/Histogram.h" //&&& +#include "util/QdispPool.h" // LSST headers #include "lsst/log/Log.h" @@ -52,33 +56,33 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.UberJob"); } -namespace lsst { namespace qserv { namespace qdisp { +namespace lsst::qserv::qdisp { UberJob::Ptr UberJob::create(Executive::Ptr const& executive, std::shared_ptr const& respHandler, int queryId, int uberJobId, qmeta::CzarId czarId, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData) { - UberJob::Ptr uJob(new UberJob(executive, respHandler, queryId, uberJobId, czarId, workerData)); + UberJob::Ptr uJob(new UberJob(executive, respHandler, queryId, uberJobId, czarId, + executive->getUjRowLimit(), workerData)); uJob->_setup(); return uJob; } UberJob::UberJob(Executive::Ptr const& executive, std::shared_ptr const& respHandler, - int queryId, int uberJobId, qmeta::CzarId czarId, + int queryId, int uberJobId, qmeta::CzarId czarId, int rowLimit, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData) - : JobBase(), - _executive(executive), + : _executive(executive), _respHandler(respHandler), _queryId(queryId), _uberJobId(uberJobId), _czarId(czarId), + _rowLimit(rowLimit), _idStr("QID=" + to_string(_queryId) + ":uj=" + to_string(uberJobId)), - _qdispPool(executive->getQdispPool()), _workerData(workerData) {} void UberJob::_setup() { - JobBase::Ptr jbPtr = shared_from_this(); - _respHandler->setJobQuery(jbPtr); + UberJob::Ptr ujPtr = shared_from_this(); + _respHandler->setUberJob(ujPtr); } bool UberJob::addJob(JobQuery::Ptr const& job) { @@ -95,73 +99,91 @@ bool UberJob::addJob(JobQuery::Ptr const& job) { return success; } -bool UberJob::runUberJob() { +util::HistogramRolling histoRunUberJob("&&&uj histoRunUberJob", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); +util::HistogramRolling histoUJSerialize("&&&uj histoUJSerialize", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + +void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelled LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj start"); // Build the uberjob payload for each job. nlohmann::json uj; unique_lock jobsLock(_jobsMtx); auto exec = _executive.lock(); - for (auto const& jqPtr : _jobs) { - jqPtr->getDescription()->incrAttemptCountScrubResultsJson(exec, true); - } // Send the uberjob to the worker auto const method = http::Method::POST; - string const url = "http://" + _wContactInfo->wHost + ":" + to_string(_wContactInfo->wPort) + "/queryjob"; + auto [ciwId, ciwHost, ciwManagment, ciwPort] = _wContactInfo->getAll(); + string const url = "http://" + ciwHost + ":" + to_string(ciwPort) + "/queryjob"; vector const headers = {"Content-Type: application/json"}; auto const& czarConfig = cconfig::CzarConfig::instance(); - // See xrdsvc::httpWorkerCzarModule::_handleQueryJob for json message parsing. - json request = {{"version", http::MetaModule::version}, - {"instance_id", czarConfig->replicationInstanceId()}, - {"auth_key", czarConfig->replicationAuthKey()}, - {"worker", _wContactInfo->wId}, - {"czar", - {{"name", czarConfig->name()}, - {"id", czarConfig->id()}, - {"management-port", czarConfig->replicationHttpPort()}, - {"management-host-name", util::get_current_host_fqdn()}}}, - {"uberjob", - {{"queryid", _queryId}, - {"uberjobid", _uberJobId}, - {"czarid", _czarId}, - {"jobs", json::array()}}}}; - - auto& jsUberJob = request["uberjob"]; - auto& jsJobs = jsUberJob["jobs"]; - for (auto const& jbPtr : _jobs) { - auto const description = jbPtr->getDescription(); - if (description == nullptr) { - throw util::Bug(ERR_LOC, cName(__func__) + " description=null for job=" + jbPtr->getIdStr()); - } - auto const jsForWorker = jbPtr->getDescription()->getJsForWorker(); - if (jsForWorker == nullptr) { - throw util::Bug(ERR_LOC, cName(__func__) + " jsForWorker=null for job=" + jbPtr->getIdStr()); + + int maxTableSizeMB = czarConfig->getMaxTableSizeMB(); + auto czInfo = protojson::CzarContactInfo::create( + czarConfig->name(), czarConfig->id(), czarConfig->replicationHttpPort(), + util::get_current_host_fqdn(), czar::Czar::czarStartupTime); + auto scanInfoPtr = exec->getScanInfo(); + + auto uberJobMsg = protojson::UberJobMsg::create( + http::MetaModule::version, czarConfig->replicationInstanceId(), czarConfig->replicationAuthKey(), + czInfo, _wContactInfo, _queryId, _uberJobId, _rowLimit, maxTableSizeMB, scanInfoPtr, _jobs); + auto startserialize = CLOCK::now(); //&&& + json request = uberJobMsg->serializeJson(); + auto endserialize = CLOCK::now(); //&&& + std::chrono::duration secsserialize = endserialize - startserialize; // &&& + histoUJSerialize.addEntry(endserialize, secsserialize.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoUJSerialize.getString("")); + + jobsLock.unlock(); // unlock so other _jobsMtx threads can advance while this waits for transmit + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj c"); + /* &&& + { // &&& testing only, delete + auto parsedReq = protojson::UberJobMsg::createFromJson(request); + json jsParsedReq = parsedReq->serializeJson(); + if (request == jsParsedReq) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj YAY!!! "); + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj noYAY request != jsParsedReq"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj request=" << request); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj jsParsedReq=" << jsParsedReq); } - json jsJob = {{"jobdesc", *jsForWorker}}; - jsJobs.push_back(jsJob); - jbPtr->getDescription()->resetJsForWorker(); // no longer needed. } - jobsLock.unlock(); // unlock so other _jobsMtx threads can advance while this waits for transmit + */ LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << request); string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " czarPost url=" << url << " request=" << request.dump() << " headers=" << headers[0]); - http::Client client(method, url, request.dump(), headers); + auto startclient = CLOCK::now(); //&&& + + auto commandHttpPool = czar::Czar::getCzar()->getCommandHttpPool(); + http::ClientConfig clientConfig; + clientConfig.httpVersion = CURL_HTTP_VERSION_1_1; // same as in qhttp + clientConfig.bufferSize = CURL_MAX_READ_SIZE; // 10 MB in the current version of libcurl + clientConfig.tcpKeepAlive = true; + clientConfig.tcpKeepIdle = 30; // the default is 60 sec + clientConfig.tcpKeepIntvl = 5; // the default is 60 sec + http::Client client(method, url, request.dump(), headers, clientConfig, commandHttpPool); bool transmitSuccess = false; string exceptionWhat; try { + //&&&util::InstanceCount ic{"runUberJob&&&"}; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj d"); json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj d1"); if (0 != response.at("success").get()) { transmitSuccess = true; } else { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " response success=0"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " ujresponse success=0"); } } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + LOGS(_log, LOG_LVL_WARN, requestContext + " ujresponse failed, ex: " + ex.what()); exceptionWhat = ex.what(); } + auto endclient = CLOCK::now(); //&&& + std::chrono::duration secsclient = endclient - startclient; // &&& + histoRunUberJob.addEntry(endclient, secsclient.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoRunUberJob.getString("")); if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure, try to send jobs elsewhere"); _unassignJobs(); // locks _jobsMtx @@ -171,7 +193,8 @@ bool UberJob::runUberJob() { } else { setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx } - return false; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj runuj end"); + return; } void UberJob::prepScrubResults() { @@ -202,8 +225,6 @@ void UberJob::_unassignJobs() { cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); } _jobs.clear(); - bool const setFlag = true; - exec->setFlagFailedUberJob(setFlag); } bool UberJob::isQueryCancelled() { @@ -243,7 +264,7 @@ bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg } void UberJob::callMarkCompleteFunc(bool success) { - LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " success=" << success); lock_guard lck(_jobsMtx); // Need to set this uberJob's status, however exec->markCompleted will set @@ -283,12 +304,11 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return _importResultError(true, "cancelled", "Query cancelled - no executive"); } - if (exec->isLimitRowComplete()) { + if (exec->isRowLimitComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, - "UberJob ignoring, enough rows already " - << "dataIgnored=" << dataIgnored); + "UberJob ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } return _importResultError(false, "rowLimited", "Enough rows already"); } @@ -301,16 +321,15 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); } - JobBase::Ptr jBaseThis = shared_from_this(); - weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); - + weak_ptr ujThis = weak_from_this(); // TODO:UJ lambda may not be the best way to do this, alsocheck synchronization - may need a mutex for // merging. - auto fileCollectFunc = [ujThis, fileUrl, rowCount](util::CmdData*) { + string const idStr = _idStr; + auto fileCollectFunc = [ujThis, fileUrl, rowCount, idStr](util::CmdData*) { auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { LOGS(_log, LOG_LVL_DEBUG, - "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); return; } uint64_t resultRows = 0; @@ -328,7 +347,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ ujPtr->_importResultFinish(resultRows); }; - auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); exec->queueFileCollect(cmd); // If the query meets the limit row complete complete criteria, it will start @@ -349,14 +368,14 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { return _workerErrorFinish(deleteData, "cancelled"); } - if (exec->isLimitRowComplete()) { + if (exec->isRowLimitComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } - return _workerErrorFinish(keepData, "none", "limitRowComplete"); + return _workerErrorFinish(keepData, "none", "rowLimitComplete"); } // Currently there are no detectable recoverable errors from workers. The only @@ -414,8 +433,15 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str return jsRet; } -nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { +void UberJob::_importResultFinish(uint64_t resultRows) { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); + return; + } + /// If this is called, the file has been collected and the worker should delete it /// /// This function should call markComplete for all jobs in the uberjob @@ -423,22 +449,16 @@ nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); if (!statusSet) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " failed to set status " << getIdStr()); - return {{"success", 0}, {"errortype", "statusMismatch"}, {"note", "failed to set status"}}; - } - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); - return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to set status, squashing " << getIdStr()); + // Something has gone very wrong + exec->squash(); + return; } bool const success = true; callMarkCompleteFunc(success); // sets status to COMPLETE exec->addResultRows(resultRows); exec->checkLimitRowComplete(); - - json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; - return jsRet; } nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, @@ -458,6 +478,36 @@ nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& e return jsRet; } +void UberJob::killUberJob() { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " stopping this UberJob and re-assigning jobs."); + + auto exec = _executive.lock(); + if (exec == nullptr || isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); + return; + } + + if (exec->isRowLimitComplete()) { + int dataIgnored = exec->incrDataIgnoredCount(); + if ((dataIgnored - 1) % 1000 == 0) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already."); + } + return; + } + + // Put this UberJob on the list of UberJobs that the worker should drop. + auto activeWorkerMap = czar::Czar::getCzar()->getActiveWorkerMap(); + auto activeWorker = activeWorkerMap->getActiveWorker(_wContactInfo->wId); + if (activeWorker != nullptr) { + activeWorker->addDeadUberJob(_queryId, _uberJobId); + } + + _unassignJobs(); + // Let Czar::_monitor reassign jobs - other UberJobs are probably being killed + // so waiting probably gets a better distribution. + return; +} + std::ostream& UberJob::dumpOS(std::ostream& os) const { os << "(jobs sz=" << _jobs.size() << "("; lock_guard lockJobsMtx(_jobsMtx); @@ -470,4 +520,12 @@ std::ostream& UberJob::dumpOS(std::ostream& os) const { return os; } -}}} // namespace lsst::qserv::qdisp +std::string UberJob::dump() const { + std::ostringstream os; + dumpOS(os); + return os.str(); +} + +std::ostream& operator<<(std::ostream& os, UberJob const& uj) { return uj.dumpOS(os); } + +} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 0dd2f69cdc..ce719d50d8 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -27,16 +27,17 @@ #include "qmeta/types.h" #include "czar/CzarChunkMap.h" // Need nested class. TODO:UJ Make non-nested? #include "czar/CzarRegistry.h" // Need nested class. TODO:UJ Make non-nested? -#include "qdisp/JobBase.h" +#include "qdisp/Executive.h" #include "qmeta/JobStatus.h" -// This header declarations +namespace lsst::qserv::util { +class QdispPool; +} + namespace lsst::qserv::qdisp { class JobQuery; -class QueryRequest; - /// This class is a contains x number of jobs that need to go to the same worker /// from a single user query, and contact information for the worker. It also holds /// some information common to all jobs. @@ -45,7 +46,7 @@ class QueryRequest; /// When this UberJobCompletes, all the Jobs it contains are registered as completed. /// If this UberJob fails, it will be destroyed, un-assigning all of its Jobs. /// Those Jobs will need to be reassigned to new UberJobs, or the query cancelled. -class UberJob : public JobBase { +class UberJob : public std::enable_shared_from_this { public: using Ptr = std::shared_ptr; @@ -57,32 +58,28 @@ class UberJob : public JobBase { UberJob(UberJob const&) = delete; UberJob& operator=(UberJob const&) = delete; - virtual ~UberJob(){}; + virtual ~UberJob() {}; + + std::string cName(const char* funcN) const { return std::string("UberJob::") + funcN + " " + getIdStr(); } bool addJob(std::shared_ptr const& job); - bool runUberJob(); - std::string cName(const char* funcN) const { return std::string("UberJob::") + funcN + " " + getIdStr(); } + /// Make a json version of this UberJob and send it to its worker. + virtual void runUberJob(); - QueryId getQueryId() const override { return _queryId; } - UberJobId getJobId() const override { + /// Kill this UberJob and unassign all Jobs so they can be used in a new UberJob if needed. + void killUberJob(); + + QueryId getQueryId() const { return _queryId; } + UberJobId getJobId() const { return _uberJobId; - } // TODO:UJ change name when JobBase no longer needed. - std::string const& getIdStr() const override { return _idStr; } - std::shared_ptr getQdispPool() override { return _qdispPool; } // TODO:UJ relocate to JobBase - std::string const& getPayload() const override { return _payload; } // TODO:UJ delete when possible. - std::shared_ptr getRespHandler() override { return _respHandler; } - std::shared_ptr getStatus() override { - return _jobStatus; - } // TODO:UJ relocate to JobBase - bool getScanInteractive() const override { return false; } ///< UberJobs are never interactive. - bool isQueryCancelled() override; // TODO:UJ relocate to JobBase - void callMarkCompleteFunc(bool success) override; ///< call markComplete for all jobs in this UberJob. - std::shared_ptr getExecutive() override { return _executive.lock(); } - - void setQueryRequest(std::shared_ptr const& qr) override { - ; // Do nothing as QueryRequest is only needed for xrootd. TODO:UJ delete function. - } + } // &&& TODO:UJ change name when JobBase no longer needed. + std::string const& getIdStr() const { return _idStr; } + std::shared_ptr getRespHandler() { return _respHandler; } + std::shared_ptr getStatus() { return _jobStatus; } + bool isQueryCancelled(); + void callMarkCompleteFunc(bool success); ///< call markComplete for all jobs in this UberJob. + std::shared_ptr getExecutive() { return _executive.lock(); } /// Return false if not ok to set the status to newState, otherwise set the state for /// this UberJob and all jobs it contains to newState. @@ -101,10 +98,12 @@ class UberJob : public JobBase { /// Set the worker information needed to send messages to the worker believed to /// be responsible for the chunks handled in this UberJob. - void setWorkerContactInfo(czar::CzarRegistry::WorkerContactInfo::Ptr const& wContactInfo) { + void setWorkerContactInfo(protojson::WorkerContactInfo::Ptr const& wContactInfo) { _wContactInfo = wContactInfo; } + protojson::WorkerContactInfo::Ptr getWorkerContactInfo() { return _wContactInfo; } + /// Get the data for the worker that should handle this UberJob. czar::CzarChunkMap::WorkerChunksData::Ptr getWorkerData() { return _workerData; } @@ -114,13 +113,16 @@ class UberJob : public JobBase { /// Handle an error from the worker. nlohmann::json workerError(int errorCode, std::string const& errorMsg); - std::ostream& dumpOS(std::ostream& os) const override; + std::ostream& dumpOS(std::ostream& os) const; + std::string dump() const; + friend std::ostream& operator<<(std::ostream& os, UberJob const& uj); -private: +protected: UberJob(std::shared_ptr const& executive, std::shared_ptr const& respHandler, - int queryId, int uberJobId, qmeta::CzarId czarId, + int queryId, int uberJobId, qmeta::CzarId czarId, int rowLimit, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); +private: /// Used to setup elements that can't be done in the constructor. void _setup(); @@ -129,7 +131,7 @@ class UberJob : public JobBase { bool _setStatusIfOk(qmeta::JobStatus::State newState, std::string const& msg); /// unassign all Jobs in this UberJob and set the Executive flag to indicate that Jobs need - /// reassignment. + /// reassignment. The list of _jobs is cleared, so multiple calls of this should be harmless. void _unassignJobs(); /// Import and error from trying to collect results. @@ -138,7 +140,7 @@ class UberJob : public JobBase { std::string const& note); /// Let the executive know that all Jobs in UberJob are complete. - nlohmann::json _importResultFinish(uint64_t resultRows); + void _importResultFinish(uint64_t resultRows); /// Let the Executive know about errors while handling results. nlohmann::json _workerErrorFinish(bool successful, std::string const& errorType = std::string(), @@ -157,15 +159,15 @@ class UberJob : public JobBase { QueryId const _queryId; UberJobId const _uberJobId; qmeta::CzarId const _czarId; + int const _rowLimit; std::string const _idStr; - std::shared_ptr _qdispPool; // TODO:UJ remove when possible. // Map of workerData czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // TODO:UJ this may not be needed // Contact information for the target worker. - czar::CzarRegistry::WorkerContactInfo::Ptr _wContactInfo; + protojson::WorkerContactInfo::Ptr _wContactInfo; // Change to ActiveWorker &&& ??? }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/XrdSsiMocks.cc b/src/qdisp/XrdSsiMocks.cc deleted file mode 100644 index bbfb243619..0000000000 --- a/src/qdisp/XrdSsiMocks.cc +++ /dev/null @@ -1,312 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - * - * @author John Gates, SLAC - */ - -// System headers -#include -#include -#include -#include -#include -#include -#include -#include - -// Third party headers -#include "XrdSsi/XrdSsiErrInfo.hh" -#include "XrdSsi/XrdSsiResponder.hh" -#include "XrdSsi/XrdSsiStream.hh" - -// LSST headers -#include "lsst/log/Log.h" -#include "proto/worker.pb.h" -#include "util/threadSafe.h" - -// Qserv headers -#include "qdisp/Executive.h" -#include "qdisp/QueryRequest.h" -#include "qdisp/XrdSsiMocks.h" - -using namespace std; - -namespace { - -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.XrdSsiMock"); - -lsst::qserv::util::FlagNotify _go(true); - -std::atomic canCount(0); -std::atomic finCount(0); -std::atomic reqCount(0); -std::atomic totCount(0); - -bool _aOK = true; - -enum RespType { RESP_BADREQ, RESP_DATA, RESP_ERROR, RESP_ERRNR, RESP_STREAM, RESP_STRERR }; - -class Agent : public XrdSsiResponder, public XrdSsiStream { -public: - void Finished(XrdSsiRequest& rqstR, XrdSsiRespInfo const& rInfo, bool cancel) override { - const char* how = (cancel ? " cancelled" : ""); - LOGS(_log, LOG_LVL_DEBUG, "Finished: " << _rNum << " rName=" << _rName << how); - _rrMutex.lock(); - UnBindRequest(); - if (cancel) canCount++; - finCount++; - _isFIN = true; - if (_active) { - _rrMutex.unlock(); - } else { - _rrMutex.unlock(); - delete this; - } - } - - void Reply(RespType rType) { - _go.wait(true); - - // We may have been cancelled before being able to reply - // - if (_isCancelled(true)) return; // we are locked now - - // Do requested reply - // - switch (rType) { - case RESP_DATA: - _ReplyData(); - break; - case RESP_ERRNR: - _reqP->doNotRetry(); - // Fallthrough - case RESP_ERROR: - _ReplyError(); - break; - case RESP_STRERR: - _noData = true; - _reqP->doNotRetry(); // Kill retries on stream errors - _ReplyStream(); - break; - default: - _reqP->doNotRetry(); - _ReplyError("Bad mock request!", 13); - break; - } - _isCancelled(false); - } - - bool SetBuff(XrdSsiErrInfo& eRef, char* buff, int blen) override { - // We may have been cancelled while waiting - // - if (_isCancelled(true)) return false; - std::thread(&Agent::_StrmResp, this, &eRef, buff, blen).detach(); - _rrMutex.unlock(); - return true; - } - - Agent(lsst::qserv::qdisp::QueryRequest* rP, std::string const& rname, int rnum) - : XrdSsiStream(XrdSsiStream::isPassive), - _reqP(rP), - _rName(rname), - _rNum(rnum), - _noData(true), - _isFIN(false), - _active(true) { - // Initialize a null message we will return as a response - // - _responseSummary = - google::protobuf::Arena::CreateMessage(_arena.get()); - lsst::qserv::proto::ResponseSummary* responseSummary = _responseSummary; - responseSummary->set_wname("localhost"); - std::string str; - responseSummary->SerializeToString(&str); - _msgBuf = str; - _bOff = 0; - _bLen = _msgBuf.size(); - } - - ~Agent() {} - -private: - bool _isCancelled(bool activate) { - if (activate) _rrMutex.lock(); - if (_isFIN) { - _rrMutex.unlock(); - delete this; - return true; - } - _active = activate; - if (!activate) _rrMutex.unlock(); - return false; - } - - void _ReplyData() { - _rspBuf = "MockResponse"; - SetResponse(_rspBuf.data(), _rspBuf.size()); - } - - void _ReplyError(const char* eMsg = "Mock Request Ignored!", int eNum = 17) { - SetErrResponse(eMsg, eNum); - } - - void _ReplyStream() { - auto stat = _setMetaData(_msgBuf.size()); - if (stat != Status::wasPosted) { - LOGS(_log, LOG_LVL_ERROR, "Agent::_ReplyStream _setMetadata failed " << stat); - } - SetResponse(this); - } - - void _StrmResp(XrdSsiErrInfo* eP, char* buff, int blen) { - std::cerr << "Stream: cleint asks for " << blen << " bytes, have " << _bLen << '\n' << std::flush; - bool last; - - // Check for cancellation while we were waiting - // - if (_isCancelled(true)) return; - - // Either reply with an error or actual data - // - if (_noData) { - blen = -17; - last = true; - eP->Set("Mock stream error!", 17); - } else { - if (_bLen <= blen) { - memcpy(buff, _msgBuf.data() + _bOff, _bLen); - blen = _bLen; - _bLen = 0; - last = true; - } else { - memcpy(buff, _msgBuf.data() + _bOff, blen); - _bOff += blen; - _bLen -= blen; - last = false; - } - } - _reqP->ProcessResponseData(*eP, buff, blen, last); - _isCancelled(false); - } - - Status _setMetaData(size_t sz) { - string str; - _responseSummary->SerializeToString(&str); - _metadata = str; - return SetMetadata(_metadata.data(), _metadata.size()); - } - - std::recursive_mutex _rrMutex; - lsst::qserv::qdisp::QueryRequest* _reqP; - std::string _rName; - std::string _rspBuf; - std::string _msgBuf; - int _bOff; - int _bLen; - int _rNum; - bool _noData; - bool _isFIN; - bool _active; - std::string _metadata; - lsst::qserv::proto::ResponseSummary* _responseSummary; - std::unique_ptr _arena{make_unique()}; -}; -} // namespace - -namespace lsst::qserv::qdisp { - -std::string XrdSsiServiceMock::_myRName; - -int XrdSsiServiceMock::getCount() { return totCount; } - -int XrdSsiServiceMock::getCanCount() { return canCount; } - -int XrdSsiServiceMock::getFinCount() { return finCount; } - -int XrdSsiServiceMock::getReqCount() { return reqCount; } - -bool XrdSsiServiceMock::isAOK() { return _aOK; } - -void XrdSsiServiceMock::Reset() { - canCount = 0; - finCount = 0; - reqCount = 0; -} - -void XrdSsiServiceMock::setGo(bool go) { _go.exchangeNotify(go); } - -void XrdSsiServiceMock::ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) { - static struct { - const char* cmd; - RespType rType; - } reqTab[] = {{"respdata", RESP_DATA}, {"resperror", RESP_ERROR}, {"resperrnr", RESP_ERRNR}, - {"respstream", RESP_STREAM}, {"respstrerr", RESP_STRERR}, {0, RESP_BADREQ}}; - - int reqNum = totCount++; - - // Check if we should verify the resource name - // - if (_myRName.size() && _myRName != resRef.rName) { - LOGS_DEBUG("Expected rname " << _myRName << " got " << resRef.rName << " from req #" << reqNum); - _aOK = false; - } - - // Get the query request object for this request and process it. - QueryRequest* r = dynamic_cast(&reqRef); - if (r) { - Agent* aP = new Agent(r, resRef.rName, reqNum); - RespType doResp; - aP->BindRequest(reqRef); - - // Get the request data and setup to handle request. Make sure the - // request string is null terminated (it should be). - // - std::string reqStr; - int reqLen; - const char* reqData = r->GetRequest(reqLen); - if (reqData != nullptr) reqStr.assign(reqData, reqLen); - reqData = reqStr.c_str(); - - // Convert request to response type - // - int i = 0; - while (reqTab[i].cmd && strcmp(reqTab[i].cmd, reqData)) i++; - if (reqTab[i].cmd) { - doResp = reqTab[i].rType; - } else { - LOGS_DEBUG("Unknown request '" << reqData << "' from req #" << reqNum); - _aOK = false; - doResp = RESP_BADREQ; - } - - // Release the request buffer (typically a no-op) - // - if (reqLen != 0) r->ReleaseRequestBuffer(); - - // Schedule a response - // - reqCount++; - std::thread(&Agent::Reply, aP, doResp).detach(); - } -} - -} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/XrdSsiMocks.h b/src/qdisp/XrdSsiMocks.h deleted file mode 100644 index 61cad5b731..0000000000 --- a/src/qdisp/XrdSsiMocks.h +++ /dev/null @@ -1,72 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - * - * @author: John Gates, SLAC (heavily modified by Andrew Hanushevsky, SLAC) - */ - -#ifndef LSST_QSERV_QDISP_XRDSSIMOCKS_H -#define LSST_QSERV_QDISP_XRDSSIMOCKS_H - -// External headers -#include "XrdSsi/XrdSsiRequest.hh" -#include "XrdSsi/XrdSsiResource.hh" -#include "XrdSsi/XrdSsiService.hh" - -// Local headers - -namespace lsst::qserv::qdisp { - -class Executive; - -/** A simplified version of XrdSsiService for testing qserv. - */ -class XrdSsiServiceMock : public XrdSsiService { -public: - void ProcessRequest(XrdSsiRequest &reqRef, XrdSsiResource &resRef) override; - - XrdSsiServiceMock(Executive *executive) {}; - - virtual ~XrdSsiServiceMock() {} - - static int getCount(); - - static int getCanCount(); - - static int getFinCount(); - - static int getReqCount(); - - static bool isAOK(); - - static void Reset(); - - static void setGo(bool go); - - static void setRName(std::string const &rname) { _myRName = rname; } - -private: - static std::string _myRName; -}; - -} // namespace lsst::qserv::qdisp - -#endif diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 5a7ffc730f..34183d0bd1 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -38,6 +38,7 @@ // Qserv headers #include "ccontrol/MergingHandler.h" #include "global/ResourceUnit.h" +#include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" #include "qdisp/QueryRequest.h" @@ -47,52 +48,146 @@ #include "qmeta/QProgressHistory.h" #include "qmeta/MessageStore.h" #include "qproc/ChunkQuerySpec.h" -#include "qproc/TaskMsgFactory.h" +#include "util/QdispPool.h" #include "util/threadSafe.h" namespace test = boost::test_tools; using namespace lsst::qserv; +using namespace std; namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.testQDisp"); } typedef util::Sequential SequentialInt; -typedef std::vector RequesterVector; +typedef vector RequesterVector; -namespace lsst::qserv::qproc { +namespace lsst::qserv::qdisp { -// Normally, there's one TaskMsgFactory that all jobs in a user query share. -// In this case, there's one MockTaskMsgFactory per job with a payload specific -// for that job. -class MockTaskMsgFactory : public TaskMsgFactory { +class ExecutiveUT; + +class TestInfo : public ResponseHandler { public: - MockTaskMsgFactory(std::string const& mockPayload_) : TaskMsgFactory(), mockPayload(mockPayload_) {} - void serializeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, - int attemptCount, qmeta::CzarId czarId, std::ostream& os) override { - os << mockPayload; + using Ptr = std::shared_ptr; + + TestInfo() {} + virtual ~TestInfo() {} + + bool goWait() { + unique_lock ulock(_infoMtx); + _infoCV.wait(ulock, [this]() { return _go == true; }); + return _ok; } - std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, std::string const& chunkResultName, - QueryId queryId, int jobId, int attemptCount, - qmeta::CzarId czarId) override { - return jsPtr; + void setGo(bool val) { + lock_guard lg(_infoMtx); + _go = val; + _infoCV.notify_all(); } - std::string mockPayload; - std::shared_ptr jsPtr; + // virtual function that won't be needed + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) override { + return {true, false}; + } + void flushHttpError(int errorCode, std::string const& errorMsg, int status) override {} + void errorFlush(std::string const& msg, int code) override {}; + Error getError() const override { return util::Error(); } + void processCancel() override {}; + void prepScrubResults(int jobId, int attempt) override {}; + + /// Print a string representation of the receiver to an ostream + std::ostream& print(std::ostream& os) const override { + os << "TestInfo ujCount=" << ujCount; + return os; + } + + atomic ujCount = 0; + +private: + bool _ok = true; + bool _go = true; + mutex _infoMtx; + condition_variable _infoCV; }; -} // namespace lsst::qserv::qproc +/// Version of UberJob specifically for this unit test. +class UberJobUT : public UberJob { +public: + using PtrUT = std::shared_ptr; + + UberJobUT(std::shared_ptr const& executive, + std::shared_ptr const& respHandler, int queryId, int uberJobId, + qmeta::CzarId czarId, int rowLimit, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData, + TestInfo::Ptr const& testInfo_) + : UberJob(executive, respHandler, queryId, uberJobId, czarId, rowLimit, workerData), + testInfo(testInfo_) {} + + void runUberJob() override { + LOGS(_log, LOG_LVL_INFO, "runUberJob() chunkId=" << chunkId); + bool ok = testInfo->goWait(); + int c = -1; + if (ok) { + c = ++(testInfo->ujCount); + } + callMarkCompleteFunc(ok); + LOGS(_log, LOG_LVL_INFO, "runUberJob() end chunkId=" << chunkId << " c=" << c); + } + + TestInfo::Ptr testInfo; + int chunkId = -1; +}; + +/// Version of Executive specifically for this unit test. +class ExecutiveUT : public Executive { +public: + using PtrUT = shared_ptr; + + ~ExecutiveUT() override = default; + + ExecutiveUT(ExecutiveConfig const& cfg, shared_ptr const& ms, + util::QdispPool::Ptr const& qdispPool, shared_ptr const& qStatus, + shared_ptr const& querySession, TestInfo::Ptr const& testInfo_) + : Executive(cfg, ms, qdispPool, qStatus, querySession), testInfo(testInfo_) {} + + void assignJobsToUberJobs() override { + vector ujVect; + + // Make an UberJobUnitTest for each job + qdisp::Executive::ChunkIdJobMapType unassignedChunks = unassignedChunksInQuery(); + for (auto const& [chunkId, jqPtr] : unassignedChunks) { + auto exec = shared_from_this(); + PtrUT execUT = dynamic_pointer_cast(exec); + auto uJob = UberJobUT::PtrUT(new UberJobUT(execUT, testInfo, getId(), ujId++, czarId, rowLimit, + targetWorker, testInfo)); + uJob->chunkId = chunkId; + uJob->addJob(jqPtr); + ujVect.push_back(uJob); + } + + for (auto const& ujPtr : ujVect) { + addAndQueueUberJob(ujPtr); + } + LOGS(_log, LOG_LVL_INFO, "assignJobsToUberJobs() end"); + } + + CzarIdType czarId = 1; + UberJobId ujId = 1; + int rowLimit = 0; + czar::CzarChunkMap::WorkerChunksData::Ptr targetWorker = nullptr; + + TestInfo::Ptr testInfo; +}; + +} // namespace lsst::qserv::qdisp qdisp::JobDescription::Ptr makeMockJobDescription(qdisp::Executive::Ptr const& ex, int sequence, ResourceUnit const& ru, std::string msg, std::shared_ptr const& mHandler) { - auto mockTaskMsgFactory = std::make_shared(msg); auto cqs = std::make_shared(); // dummy, unused in this case. std::string chunkResultName = "dummyResultTableName"; qmeta::CzarId const czarId = 1; - auto job = qdisp::JobDescription::create(czarId, ex->getId(), sequence, ru, mHandler, mockTaskMsgFactory, + auto job = qdisp::JobDescription::create(czarId, ex->getId(), sequence, ru, mHandler, cqs, chunkResultName, true); return job; } @@ -101,13 +196,15 @@ qdisp::JobDescription::Ptr makeMockJobDescription(qdisp::Executive::Ptr const& e // that we return a shared pointer to the last constructed JobQuery object. // This only makes sense for single query jobs. // + std::shared_ptr addMockRequests(qdisp::Executive::Ptr const& ex, SequentialInt& sequence, - int chunkID, std::string msg, RequesterVector& rv) { - ResourceUnit ru; + int startingChunkId, std::string msg, RequesterVector& rv) { std::shared_ptr jobQuery; int copies = rv.size(); - ru.setAsDbChunk("Mock", chunkID); for (int j = 0; j < copies; ++j) { + ResourceUnit ru; + int chunkId = startingChunkId + j; + ru.setAsDbChunk("Mock", chunkId); // The job copies the JobDescription. qdisp::JobDescription::Ptr job = makeMockJobDescription(ex, sequence.incr(), ru, msg, rv[j]); jobQuery = ex->add(job); @@ -115,12 +212,9 @@ std::shared_ptr addMockRequests(qdisp::Executive::Ptr const& ex return jobQuery; } -/** Start adds 'copies' number of test requests that each sleep for 'millisecs' time - * before signaling to 'ex' that they are done. - * Returns time to complete in seconds. - */ -std::shared_ptr executiveTest(qdisp::Executive::Ptr const& ex, SequentialInt& sequence, +std::shared_ptr executiveTest(qdisp::ExecutiveUT::PtrUT const& ex, SequentialInt& sequence, int chunkId, std::string msg, int copies) { + LOGS(_log, LOG_LVL_INFO, "executiveTest start"); // Test class Executive::add // Modeled after ccontrol::UserQuery::submit() ResourceUnit ru; @@ -132,14 +226,17 @@ std::shared_ptr executiveTest(qdisp::Executive::Ptr const& ex, for (int j = 0; j < copies; ++j) { rv.push_back(mh); } - return addMockRequests(ex, sequence, chunkId, msg, rv); + auto ret = addMockRequests(ex, sequence, chunkId, msg, rv); + ex->assignJobsToUberJobs(); + LOGS(_log, LOG_LVL_INFO, "executiveTest end"); + return ret; } /** This function is run in a separate thread to fail the test if it takes too long * for the jobs to complete. */ void timeoutFunc(std::atomic& flagDone, int millisecs) { - LOGS_DEBUG("timeoutFunc"); + LOGS_INFO("timeoutFunc"); int total = 0; bool done = flagDone; int maxTime = millisecs * 1000; @@ -148,7 +245,7 @@ void timeoutFunc(std::atomic& flagDone, int millisecs) { total += sleepTime; usleep(sleepTime); done = flagDone; - LOGS_DEBUG("timeoutFunc done=" << done << " total=" << total); + LOGS_INFO("timeoutFunc done=" << done << " total=" << total); } LOGS_ERROR("timeoutFunc done=" << done << " total=" << total << " timedOut=" << (total >= maxTime)); BOOST_REQUIRE(done == true); @@ -163,26 +260,25 @@ class SetupTest { std::string str; qdisp::ExecutiveConfig::Ptr conf; std::shared_ptr ms; - qdisp::QdispPool::Ptr qdispPool; - qdisp::SharedResources::Ptr sharedResources; - qdisp::Executive::Ptr ex; + util::QdispPool::Ptr qdispPool; + qdisp::ExecutiveUT::PtrUT ex; std::shared_ptr jqTest; // used only when needed - boost::asio::io_service asioIoService; + qdisp::TestInfo::Ptr testInfo = qdisp::TestInfo::Ptr(new qdisp::TestInfo()); - SetupTest(const char* request) { + SetupTest(const char* request, util::QdispPool::Ptr const& qPool_) : qdispPool(qPool_) { + LOGS(_log, LOG_LVL_INFO, "SetupTest start"); qrMsg = request; - qdisp::XrdSsiServiceMock::Reset(); str = qdisp::ExecutiveConfig::getMockStr(); conf = std::make_shared(str, 0); // No updating of QMeta. ms = std::make_shared(); - qdispPool = std::make_shared(true); - sharedResources = qdisp::SharedResources::create(qdispPool); - + auto tInfo = qdisp::TestInfo::Ptr(new qdisp::TestInfo()); std::shared_ptr queryProgress; // No updating QProgress, nullptr std::shared_ptr queryProgressHistory; // No updating QProgressHistory, nullptr - ex = qdisp::Executive::create(*conf, ms, sharedResources, queryProgress, queryProgressHistory, - nullptr, asioIoService); + ex = qdisp::ExecutiveUT::PtrUT( + new qdisp::ExecutiveUT(*conf, ms, qdispPool, qProgress, + queryProgressHistory, nullptr, testInfo)); + LOGS(_log, LOG_LVL_INFO, "SetupTest end"); } ~SetupTest() {} }; @@ -196,7 +292,19 @@ BOOST_AUTO_TEST_SUITE(Suite) int chunkId = 1234; int millisInt = 50000; +util::QdispPool::Ptr globalQdispPool; +qdisp::CzarStats::Ptr globalCzarStats; + BOOST_AUTO_TEST_CASE(Executive) { + int qPoolSize = 1000; + int maxPriority = 2; + vector vectRunSizes = {50, 50, 50, 50}; + vector vectMinRunningSizes = {0, 1, 3, 3}; + globalQdispPool = util::QdispPool::Ptr( + new util::QdispPool(qPoolSize, maxPriority, vectRunSizes, vectMinRunningSizes)); + qdisp::CzarStats::setup(globalQdispPool); + globalCzarStats = qdisp::CzarStats::get(); + // Variables for all executive sub-tests. Note that all executive tests // are full roundtrip tests. So, if these succeed then it's likely all // other query tests will succeed. So, much of this is redundant. @@ -205,63 +313,55 @@ BOOST_AUTO_TEST_CASE(Executive) { int jobs = 0; _log.setLevel(LOG_LVL_DEBUG); // Ugly but boost test suite forces this std::thread timeoutT(&timeoutFunc, std::ref(done), millisInt); - qdisp::XrdSsiServiceMock::setRName("/chk/Mock/1234"); // Test single instance { - LOGS_DEBUG("Executive single query test"); - SetupTest tEnv("respdata"); + LOGS_INFO("Executive single query test"); + SetupTest tEnv("respdata", globalQdispPool); SequentialInt sequence(0); tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); jobs = 1; - LOGS_DEBUG("jobs=1"); + LOGS_INFO("jobs=1"); tEnv.ex->join(); - LOGS_DEBUG("Executive single query test checking"); + LOGS_INFO("Executive single query test checking"); BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::COMPLETE); BOOST_CHECK(tEnv.ex->getEmpty() == true); } // Test 4 jobs { - LOGS_DEBUG("Executive four parallel jobs test"); - SetupTest tEnv("respdata"); + LOGS_INFO("Executive four parallel jobs test"); + SetupTest tEnv("respdata", globalQdispPool); SequentialInt sequence(0); executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 4); jobs += 4; - LOGS_DEBUG("ex->joining()"); + LOGS_INFO("ex->joining()"); tEnv.ex->join(); - LOGS_DEBUG("Executive four parallel jobs test checking"); + LOGS_INFO("Executive four parallel jobs test checking"); BOOST_CHECK(tEnv.ex->getEmpty() == true); } // Test that we can detect ex._empty == false. { - LOGS_DEBUG("Executive detect non-empty job queue test"); - SetupTest tEnv("respdata"); + LOGS_INFO("Executive detect non-empty job queue test"); + SetupTest tEnv("respdata", globalQdispPool); SequentialInt sequence(0); - qdisp::XrdSsiServiceMock::setGo(false); executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 5); jobs += 5; - while (qdisp::XrdSsiServiceMock::getCount() < jobs) { - LOGS_DEBUG("waiting for _count(" << qdisp::XrdSsiServiceMock::getCount() << ") == jobs(" << jobs - << ")"); - usleep(10000); - } BOOST_CHECK(tEnv.ex->getEmpty() == false); - qdisp::XrdSsiServiceMock::setGo(true); - LOGS_DEBUG("ex->joining()"); + LOGS_INFO("ex->joining()"); tEnv.ex->join(); - LOGS_DEBUG("ex->join() joined"); + LOGS_INFO("ex->join() joined"); BOOST_CHECK(tEnv.ex->getEmpty() == true); } done = true; timeoutT.join(); - LOGS_DEBUG("Executive test end"); + LOGS_INFO("Executive test end"); } BOOST_AUTO_TEST_CASE(MessageStore) { - LOGS_DEBUG("MessageStore test start"); + LOGS_INFO("MessageStore test start"); qmeta::MessageStore ms; BOOST_CHECK(ms.messageCount() == 0); ms.addMessage(123, "EXECUTIVE", 456, "test1"); @@ -272,110 +372,37 @@ BOOST_AUTO_TEST_CASE(MessageStore) { BOOST_CHECK(ms.messageCount(-12) == 2); qmeta::QueryMessage qm = ms.getMessage(1); BOOST_CHECK(qm.chunkId == 124 && qm.code == -12 && str.compare(qm.description) == 0); - LOGS_DEBUG("MessageStore test end"); -} - -BOOST_AUTO_TEST_CASE(QueryRequest) { - { - LOGS_DEBUG("QueryRequest error retry test"); - // Setup Executive and for retry test when receiving an error - // Note executive maps RESPONSE_ERROR to RESULT_ERROR - SetupTest tEnv("resperror"); - SequentialInt sequence(0); - tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); - tEnv.ex->join(); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::RESULT_ERROR); - BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() > 1); // Retried, eh? - BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == qdisp::XrdSsiServiceMock::getReqCount()); - } - - { - LOGS_DEBUG("QueryRequest error noretry test 2"); - // Setup Executive and for no retry test when receiving an error - // Note executive maps RESPONSE_ERROR to RESULT_ERROR - SetupTest tEnv("resperrnr"); - SequentialInt sequence(0); - tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); - tEnv.ex->join(); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::RESULT_ERROR); - BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == 1); - } - - { - LOGS_DEBUG("QueryRequest stream with data error test"); - // Setup Executive and for no retry test when receiving an error - // Note executive maps RESPONSE_DATA_NACK to RESULT_ERROR - SetupTest tEnv("respstrerr"); - SequentialInt sequence(0); - tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); - tEnv.ex->join(); - LOGS_DEBUG("tEnv.jqTest->...state = " << tEnv.jqTest->getStatus()->getInfo().state); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == qmeta::JobStatus::RESULT_ERROR); - BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == 1); // No retries! - } - - // We wish we could do the stream response with no results test but the - // needed information is too complex to figure out (well, one day we will). - // So, we've commented this out but the framework exists modulo the needed - // responses (see XrdSsiMocks::Agent). So, this gets punted into the - // integration test (too bad). - /* - { - LOGS_DEBUG("QueryRequest stream with no results test"); - SetupTest tEnv("respstream"); - SequentialInt sequence(0); - tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); - tEnv.ex->join(); - BOOST_CHECK(tEnv.jqTest->getStatus()->getInfo().state == - qmeta::JobStatus::COMPLETE); - BOOST_CHECK(qdisp::XrdSsiServiceMock::getFinCount() == 1); - } - */ - LOGS_DEBUG("QueryRequest test end"); + LOGS_INFO("MessageStore test end"); } BOOST_AUTO_TEST_CASE(ExecutiveCancel) { // Test that aJobQuery can be cancelled and ends in correct state // { - LOGS_DEBUG("ExecutiveCancel: squash it test"); - SetupTest tEnv("respdata"); - qdisp::XrdSsiServiceMock::setGo(false); // Can't let jobs run or they are untracked before squash + LOGS_INFO("ExecutiveCancel: squash it test"); + SetupTest tEnv("respdata", globalQdispPool); + tEnv.testInfo->setGo(false); // Can't let jobs run or they are untracked before + // squash SequentialInt sequence(0); tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); tEnv.ex->squash(); - qdisp::XrdSsiServiceMock::setGo(true); usleep(250000); // Give mock threads a quarter second to complete. tEnv.ex->join(); BOOST_CHECK(tEnv.jqTest->isQueryCancelled() == true); - // Note that the query might not have actually called ProcessRequest() - // but if it did, then it must have called Finished() with cancel. - // - BOOST_CHECK(qdisp::XrdSsiServiceMock::getCanCount() == qdisp::XrdSsiServiceMock::getReqCount()); } // Test that multiple JobQueries are cancelled. { - LOGS_DEBUG("ExecutiveCancel: squash 20 test"); - SetupTest tEnv("respdata"); - qdisp::XrdSsiServiceMock::setGo(false); // Can't let jobs run or they are untracked before squash + LOGS_INFO("ExecutiveCancel: squash 20 test"); + SetupTest tEnv("respdata", globalQdispPool); + // squash SequentialInt sequence(0); executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 20); tEnv.ex->squash(); tEnv.ex->squash(); // check that squashing twice doesn't cause issues. - qdisp::XrdSsiServiceMock::setGo(true); - usleep(250000); // Give mock threads a quarter second to complete. + usleep(250000); // Give mock threads a quarter second to complete. tEnv.ex->join(); - // Note that the cancel count might not be 20 as some queries will cancel - // themselves before they get around to issuing ProcessRequest(). - // - BOOST_CHECK(qdisp::XrdSsiServiceMock::getCanCount() == qdisp::XrdSsiServiceMock::getReqCount()); } } -BOOST_AUTO_TEST_CASE(ServiceMock) { - // Verify that our service object did not see anything unusual. - BOOST_CHECK(qdisp::XrdSsiServiceMock::isAOK()); -} - BOOST_AUTO_TEST_SUITE_END() diff --git a/src/qhttp/Server.cc b/src/qhttp/Server.cc index 0cb23ae7ea..5116255cb6 100644 --- a/src/qhttp/Server.cc +++ b/src/qhttp/Server.cc @@ -277,31 +277,7 @@ std::shared_ptr Server::_findPathHandler(Request::Ptr reque for (auto& pathHandler : pathHandlersIt->second) { if (boost::regex_match(request->path, pathMatch, pathHandler.path.regex)) { pathHandler.path.updateParamsFromMatch(request, pathMatch); -#if 0 // &&& <<<<<<< HEAD return std::make_shared(pathHandler); -#else // &&& ===== - LOGLS_DEBUG(_log, logger(this) << logger(request->_socket) << "invoking handler for " - << pathHandler.path.regex); - try { - pathHandler.handler(request, response); - } catch (boost::system::system_error const& e) { - LOGLS_ERROR(_log, logger(this) << logger(request->_socket) - << "exception thrown from handler: " << e.what()); - switch (e.code().value()) { - case errc::permission_denied: - response->sendStatus(STATUS_FORBIDDEN); - break; - default: - response->sendStatus(STATUS_INTERNAL_SERVER_ERR); - break; - } - } catch (std::exception const& e) { - LOGLS_ERROR(_log, logger(this) << logger(request->_socket) - << "exception thrown from handler: " << e.what()); - response->sendStatus(STATUS_INTERNAL_SERVER_ERR); - } - return; -#endif //&&& >>>>>>> ca9f7b24f (Added some error handling.) } } } diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index 0269824fb1..c3bab4dc62 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -43,6 +43,7 @@ #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" #include "sql/SqlResults.h" +#include "util/TimeUtils.h" using namespace std; @@ -780,6 +781,8 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c // Check if the table needs to be read. Note that the default value of // the previous update timestamp always forces an attempt to read the map. auto const updateTime = _getChunkMapUpdateTime(lock); + LOGS(_log, LOG_LVL_INFO, + "QMetaMysql::getChunkMap updateTime=" << util::TimeUtils::timePointToDateTimeString(updateTime)); bool const force = (prevUpdateTime == chrono::time_point()) || (prevUpdateTime < updateTime); if (!force) { @@ -827,8 +830,9 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock sql::SqlErrorObject errObj; sql::SqlResults results; string const tableName = "chunkMapStatus"; - string const query = - "SELECT TIME_TO_SEC(`update_time`) FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; + string const query = "SELECT UNIX_TIMESTAMP(`update_time`) FROM `" + tableName + + "` ORDER BY `update_time` DESC LIMIT 1"; + LOGS(_log, LOG_LVL_DEBUG, "Executing query: " << query); if (!_conn->runQuery(query, results, errObj)) { LOGS(_log, LOG_LVL_ERROR, "query failed: " << query); @@ -845,6 +849,7 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock throw ConsistencyError(ERR_LOC, "Too many rows in result set of query " + query); } try { + LOGS(_log, LOG_LVL_TRACE, "QMetaMysql::_getChunkMapUpdateTime " << updateTime[0]); return chrono::time_point() + chrono::seconds(stol(updateTime[0])); } catch (exception const& ex) { string const msg = "Failed to parse result set of query " + query + ", ex: " + string(ex.what()); diff --git a/src/qmeta/types.h b/src/qmeta/types.h index 28e8338fa4..6f55562b3b 100644 --- a/src/qmeta/types.h +++ b/src/qmeta/types.h @@ -38,7 +38,7 @@ namespace lsst::qserv::qmeta { */ /// Typedef for Czar ID in query metadata. -typedef std::uint32_t CzarId; +typedef CzarIdType CzarId; // uint32_t TODO:UJ Replace qmeta::CzarId with global } // namespace lsst::qserv::qmeta diff --git a/src/qproc/CMakeLists.txt b/src/qproc/CMakeLists.txt index e95b72e0b2..e87e0b3979 100644 --- a/src/qproc/CMakeLists.txt +++ b/src/qproc/CMakeLists.txt @@ -8,7 +8,6 @@ target_sources(qproc PRIVATE IndexMap.cc QuerySession.cc SecondaryIndex.cc - TaskMsgFactory.cc ) target_link_libraries(qproc PRIVATE @@ -36,7 +35,6 @@ FUNCTION(qproc_tests) css qmeta rproc - xrdreq Boost::unit_test_framework Threads::Threads ) diff --git a/src/qproc/ChunkQuerySpec.h b/src/qproc/ChunkQuerySpec.h index a8e7cdc643..41582368f0 100644 --- a/src/qproc/ChunkQuerySpec.h +++ b/src/qproc/ChunkQuerySpec.h @@ -39,7 +39,7 @@ // Qserv headers #include "global/DbTable.h" #include "global/stringTypes.h" -#include "proto/ScanTableInfo.h" +#include "protojson/ScanTableInfo.h" namespace lsst::qserv::qproc { @@ -52,20 +52,23 @@ class ChunkQuerySpec { using Ptr = std::shared_ptr; ChunkQuerySpec() {} - ChunkQuerySpec(std::string const& db_, int chunkId_, proto::ScanInfo const& scanInfo_, + //&&&ChunkQuerySpec(std::string const& db_, int chunkId_, protojson::ScanInfo const& scanInfo_, + ChunkQuerySpec(std::string const& db_, int chunkId_, protojson::ScanInfo::Ptr const& scanInfo_, bool scanInteractive_) : db(db_), chunkId(chunkId_), scanInfo(scanInfo_), scanInteractive(scanInteractive_) {} // Contents could change std::string db{""}; ///< dominant db int chunkId{0}; - proto::ScanInfo scanInfo; ///< shared-scan candidates + //&&&protojson::ScanInfo scanInfo; ///< shared-scan candidates + protojson::ScanInfo::Ptr scanInfo; ///< shared-scan candidates // Consider saving subChunkTable templates, and substituting the chunkIds // and subChunkIds into them on-the-fly. bool scanInteractive{false}; DbTableSet subChunkTables; std::vector subChunkIds; - std::vector queries; + std::vector queries; // &&& remove if possible + std::vector queryTemplates; // Consider promoting the concept of container of ChunkQuerySpec // in the hopes of increased code cleanliness. std::shared_ptr nextFragment; ///< ad-hoc linked list (consider removal) diff --git a/src/qproc/ChunkSpec.cc b/src/qproc/ChunkSpec.cc index 1bd36261fb..fa9a8132ff 100644 --- a/src/qproc/ChunkSpec.cc +++ b/src/qproc/ChunkSpec.cc @@ -44,7 +44,15 @@ namespace { // File-scope helpers /// A "good" number of subchunks to include in a chunk query. This is /// a guess. The best value is an open question -int const GOOD_SUBCHUNK_COUNT = 20; +// TODO:UJ `ChunkSpecFragmenter` has the purpose of limiting the +// number of subchunks per ChunkSpec (which works out to +// subchunkids per Job). +// Each subchunk gets its own task on the worker, so this +// is probably no longer helpful. Making the limit absurdly +// high should have the effect of disabling the code +// while checking if there are unexpected side effects. +// int const GOOD_SUBCHUNK_COUNT = 20; +int const GOOD_SUBCHUNK_COUNT = 2'000'000; } // namespace namespace lsst::qserv::qproc { diff --git a/src/qproc/ChunkSpec.h b/src/qproc/ChunkSpec.h index 9bf31053ee..777cd9d87f 100644 --- a/src/qproc/ChunkSpec.h +++ b/src/qproc/ChunkSpec.h @@ -93,6 +93,8 @@ ChunkSpecVector intersect(ChunkSpecVector const& a, ChunkSpecVector const& b); void normalize(ChunkSpecVector& specs); /// An iterating fragmenter to reduce the number of subChunkIds per ChunkSpec +/// TODO:UJ Fragmenting the the Jobs probably no longer makes sense, see +/// `GOOD_SUBCHUNK_COUNT` definition. class ChunkSpecFragmenter { public: ChunkSpecFragmenter(ChunkSpec const& s); diff --git a/src/qproc/QuerySession.cc b/src/qproc/QuerySession.cc index b5fda17ae1..9bd643265b 100644 --- a/src/qproc/QuerySession.cc +++ b/src/qproc/QuerySession.cc @@ -363,8 +363,8 @@ void QuerySession::print(std::ostream& os) const { os << " needs merge: " << this->needsMerge(); os << " 1st parallel statement: \"" << par << "\""; os << " merge statement: \"" << mer << "\""; - os << " scanRating:" << _context->scanInfo.scanRating; - for (auto const& tbl : _context->scanInfo.infoTables) { + os << " scanRating:" << _context->scanInfo->scanRating; + for (auto const& tbl : _context->scanInfo->infoTables) { os << " ScanTable: " << tbl.db << "." << tbl.table << " lock=" << tbl.lockInMemory << " rating=" << tbl.scanRating; } @@ -391,6 +391,7 @@ std::vector QuerySession::_buildChunkQueries(query::QueryTemplate:: } for (auto&& queryTemplate : queryTemplates) { + LOGS(_log, LOG_LVL_WARN, "&&&uj QuerySession::_buildChunkQueries qt=" << queryTemplate.dump()); std::string str = _context->queryMapping->apply(chunkSpec, queryTemplate); chunkQueries.push_back(std::move(str)); } @@ -402,6 +403,8 @@ std::ostream& operator<<(std::ostream& out, QuerySession const& querySession) { return out; } +protojson::ScanInfo::Ptr QuerySession::getScanInfo() const { return _context->scanInfo; } + ChunkQuerySpec::Ptr QuerySession::buildChunkQuerySpec(query::QueryTemplate::Vect const& queryTemplates, ChunkSpec const& chunkSpec, bool fillInChunkIdTag) const { diff --git a/src/qproc/QuerySession.h b/src/qproc/QuerySession.h index a85634267c..a368abc060 100644 --- a/src/qproc/QuerySession.h +++ b/src/qproc/QuerySession.h @@ -175,6 +175,8 @@ class QuerySession { void setScanInteractive(); bool getScanInteractive() const { return _scanInteractive; } + protojson::ScanInfo::Ptr getScanInfo() const; + /** * Print query session to stream. * diff --git a/src/qproc/TaskMsgFactory.cc b/src/qproc/TaskMsgFactory.cc deleted file mode 100644 index 8a2d7434dc..0000000000 --- a/src/qproc/TaskMsgFactory.cc +++ /dev/null @@ -1,308 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2017 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -/** - * @file - * - * @brief TaskMsgFactory is a factory for TaskMsg (protobuf) objects. - * - * @author Daniel L. Wang, SLAC - */ - -// Class header -#include "qproc/TaskMsgFactory.h" - -// System headers -#include - -// Third-party headers -#include "nlohmann/json.hpp" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "cconfig/CzarConfig.h" -#include "global/intTypes.h" -#include "qmeta/types.h" -#include "qproc/ChunkQuerySpec.h" -#include "util/common.h" - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qproc.TaskMsgFactory"); -} - -using namespace std; - -namespace lsst::qserv::qproc { - -// TODO:UJ - Probaly just delete this -bool TaskMsgFactory::fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& chunkQuerySpec, - std::string const& chunkResultName, QueryId queryId, int jobId, - int attemptCount, qmeta::CzarId czarId) { - std::string resultTable("Asdfasfd"); - if (!chunkResultName.empty()) { - resultTable = chunkResultName; - } - // shared - taskMsg->set_db(chunkQuerySpec.db); - taskMsg->set_queryid(queryId); - taskMsg->set_jobid(jobId); - taskMsg->set_attemptcount(attemptCount); - taskMsg->set_czarid(czarId); - - // scanTables (for shared scans) - // check if more than 1 db in scanInfo - std::string db; - for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { - if (db.empty()) { - db = sTbl.db; - } - } - - for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { - lsst::qserv::proto::TaskMsg_ScanTable* msgScanTbl = taskMsg->add_scantable(); - sTbl.copyToScanTable(msgScanTbl); - } - - taskMsg->set_scanpriority(chunkQuerySpec.scanInfo.scanRating); - taskMsg->set_scaninteractive(chunkQuerySpec.scanInteractive); - - // per-chunk - taskMsg->set_chunkid(chunkQuerySpec.chunkId); - // per-fragment - // TODO refactor to simplify - if (chunkQuerySpec.nextFragment.get()) { - ChunkQuerySpec const* sPtr = &chunkQuerySpec; - while (sPtr) { - LOGS(_log, LOG_LVL_TRACE, "nextFragment"); - for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { - LOGS(_log, LOG_LVL_TRACE, (sPtr->queries).at(t)); - } - // Linked fragments will not have valid subChunkTables vectors, - // So, we reuse the root fragment's vector. - _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, - sPtr->queries); - sPtr = sPtr->nextFragment.get(); - } - } else { - LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); - for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { - LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); - } - _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, - chunkQuerySpec.queries); - } - return true; -} - -std::shared_ptr TaskMsgFactory::_makeMsg(ChunkQuerySpec const& chunkQuerySpec, - std::string const& chunkResultName, QueryId queryId, - int jobId, int attemptCount, qmeta::CzarId czarId) { - std::string resultTable("Asdfasfd"); - if (!chunkResultName.empty()) { - resultTable = chunkResultName; - } - auto taskMsg = std::make_shared(); - // shared - taskMsg->set_db(chunkQuerySpec.db); - taskMsg->set_queryid(queryId); - taskMsg->set_jobid(jobId); - taskMsg->set_attemptcount(attemptCount); - taskMsg->set_czarid(czarId); - // scanTables (for shared scans) - // check if more than 1 db in scanInfo - std::string db; - for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { - if (db.empty()) { - db = sTbl.db; - } - } - - for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { - lsst::qserv::proto::TaskMsg_ScanTable* msgScanTbl = taskMsg->add_scantable(); - sTbl.copyToScanTable(msgScanTbl); - } - - taskMsg->set_scanpriority(chunkQuerySpec.scanInfo.scanRating); - taskMsg->set_scaninteractive(chunkQuerySpec.scanInteractive); - taskMsg->set_maxtablesize_mb(cconfig::CzarConfig::instance()->getMaxTableSizeMB()); - - // per-chunk - taskMsg->set_chunkid(chunkQuerySpec.chunkId); - // per-fragment - // TODO refactor to simplify - if (chunkQuerySpec.nextFragment.get()) { - ChunkQuerySpec const* sPtr = &chunkQuerySpec; - while (sPtr) { - LOGS(_log, LOG_LVL_TRACE, "nextFragment"); - for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { - LOGS(_log, LOG_LVL_TRACE, (sPtr->queries).at(t)); - } - // Linked fragments will not have valid subChunkTables vectors, - // So, we reuse the root fragment's vector. - _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, - sPtr->queries); - sPtr = sPtr->nextFragment.get(); - } - } else { - LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); - for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { - LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); - } - _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, - chunkQuerySpec.queries); - } - return taskMsg; -} - -void TaskMsgFactory::_addFragment(proto::TaskMsg& taskMsg, std::string const& resultName, - DbTableSet const& subChunkTables, std::vector const& subChunkIds, - std::vector const& queries) { - proto::TaskMsg::Fragment* frag = taskMsg.add_fragment(); - frag->set_resulttable(resultName); - - for (auto& qry : queries) { - frag->add_query(qry); - } - - proto::TaskMsg_Subchunk sc; - - // Add the db+table pairs to the subchunk. - for (auto& tbl : subChunkTables) { - proto::TaskMsg_Subchunk_DbTbl* dbTbl = sc.add_dbtbl(); - dbTbl->set_db(tbl.db); - dbTbl->set_tbl(tbl.table); - LOGS(_log, LOG_LVL_TRACE, "added dbtbl=" << tbl.db << "." << tbl.table); - } - - for (auto& subChunkId : subChunkIds) { - sc.add_id(subChunkId); - } - - frag->mutable_subchunks()->CopyFrom(sc); -} - -void TaskMsgFactory::serializeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, - QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId, - std::ostream& os) { - std::shared_ptr m = _makeMsg(s, chunkResultName, queryId, jobId, attemptCount, czarId); - m->SerializeToOstream(&os); -} - -std::shared_ptr TaskMsgFactory::makeMsgJson(ChunkQuerySpec const& chunkQuerySpec, - std::string const& chunkResultName, - QueryId queryId, int jobId, int attemptCount, - qmeta::CzarId czarId) { - std::string resultTable("Asdfasfd"); - if (!chunkResultName.empty()) { - resultTable = chunkResultName; - } - - // TODO:UJ verify that these can be put in the uberjob to reduce duplicates - // and the size of the message. - auto jsJobMsgPtr = std::shared_ptr( - new nlohmann::json({{"czarId", czarId}, - {"queryId", queryId}, - {"jobId", jobId}, - {"attemptCount", attemptCount}, - {"querySpecDb", chunkQuerySpec.db}, - {"scanPriority", chunkQuerySpec.scanInfo.scanRating}, - {"scanInteractive", chunkQuerySpec.scanInteractive}, - {"maxTableSize", (cconfig::CzarConfig::instance()->getMaxTableSizeMB())}, - {"chunkScanTables", nlohmann::json::array()}, - {"chunkId", chunkQuerySpec.chunkId}, - {"queryFragments", nlohmann::json::array()}})); - - auto& jsJobMsg = *jsJobMsgPtr; - - auto& chunkScanTables = jsJobMsg["chunkScanTables"]; - for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { - nlohmann::json cst = {{"db", sTbl.db}, - {"table", sTbl.table}, - {"lockInMemory", sTbl.lockInMemory}, - {"tblScanRating", sTbl.scanRating}}; - chunkScanTables.push_back(move(cst)); - } - - auto& jsFragments = jsJobMsg["queryFragments"]; - if (chunkQuerySpec.nextFragment.get()) { - ChunkQuerySpec const* sPtr = &chunkQuerySpec; - while (sPtr) { - LOGS(_log, LOG_LVL_TRACE, "nextFragment"); - for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " q=" << (sPtr->queries).at(t)); - } - for (auto const& sbi : sPtr->subChunkIds) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " sbi=" << sbi); - } - // Linked fragments will not have valid subChunkTables vectors, - // So, we reuse the root fragment's vector. - _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, - sPtr->queries); - sPtr = sPtr->nextFragment.get(); - } - } else { - LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); - for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { - LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); - } - _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, - chunkQuerySpec.queries); - } - - return jsJobMsgPtr; -} - -void TaskMsgFactory::_addFragmentJson(nlohmann::json& jsFragments, std::string const& resultName, - DbTableSet const& subChunkTables, std::vector const& subchunkIds, - std::vector const& queries) { - nlohmann::json jsFrag = {{"resultTable", resultName}, - {"queries", nlohmann::json::array()}, - {"subchunkTables", nlohmann::json::array()}, - {"subchunkIds", nlohmann::json::array()}}; - - auto& jsQueries = jsFrag["queries"]; - for (auto& qry : queries) { - nlohmann::json jsQry = {{"subQuery", qry}}; - jsQueries.push_back(move(jsQry)); - } - - // Add the db+table pairs to the subchunk. - auto& jsSubchunkTables = jsFrag["subchunkTables"]; - for (auto& tbl : subChunkTables) { - nlohmann::json jsSubchunkTbl = {{"scDb", tbl.db}, {"scTable", tbl.table}}; - jsSubchunkTables.push_back(move(jsSubchunkTbl)); - LOGS(_log, LOG_LVL_TRACE, "added dbtbl=" << tbl.db << "." << tbl.table); - } - - // Add subchunk id numbers - auto& jsSubchunkIds = jsFrag["subchunkIds"]; - for (auto& subchunkId : subchunkIds) { - jsSubchunkIds.push_back(subchunkId); - } - - jsFragments.push_back(move(jsFrag)); -} - -} // namespace lsst::qserv::qproc diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h deleted file mode 100644 index d770d2c5c4..0000000000 --- a/src/qproc/TaskMsgFactory.h +++ /dev/null @@ -1,93 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2017 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_QPROC_TASKMSGFACTORY_H -#define LSST_QSERV_QPROC_TASKMSGFACTORY_H -/** - * @file - * - * @brief TaskMsgFactory is a factory for TaskMsg (protobuf) objects. - * - * @author Daniel L. Wang, SLAC - */ - -// System headers -#include -#include - -// Third party headers -#include "nlohmann/json.hpp" - -// Qserv headers -#include "global/DbTable.h" -#include "global/intTypes.h" -#include "proto/worker.pb.h" -#include "qmeta/types.h" - -namespace lsst::qserv::qproc { - -class ChunkQuerySpec; - -/// TaskMsgFactory is a factory for TaskMsg (protobuf) objects. -/// All member variables must be thread safe. -class TaskMsgFactory { -public: - using Ptr = std::shared_ptr; - - TaskMsgFactory() = default; - virtual ~TaskMsgFactory() {} - - /// Construct a TaskMsg and serialize it to a stream - virtual void serializeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, - int jobId, int attemptCount, qmeta::CzarId czarId, std::ostream& os); - - /// Use the provided information to fill in taskMsg. - /// @return true if successful. - bool fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& s, std::string const& chunkResultName, - QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId); - - /// Make and return the json message for a single Job. - virtual std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, - std::string const& chunkResultName, QueryId queryId, - int jobId, int attemptCount, qmeta::CzarId czarId); - -private: - // TODO:UJ delete when possible - std::shared_ptr _makeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, - QueryId queryId, int jobId, int attemptCount, - qmeta::CzarId czarId); - - // TODO:UJ delete when possible - void _addFragment(proto::TaskMsg& taskMsg, std::string const& resultName, - DbTableSet const& subChunkTables, std::vector const& subChunkIds, - std::vector const& queries); - - /// Make a json message for a single fragment. - void _addFragmentJson(nlohmann::json& jsFragments, std::string const& resultName, - DbTableSet const& subChunkTables, std::vector const& subChunkIds, - std::vector const& queries); -}; - -} // namespace lsst::qserv::qproc - -#endif // LSST_QSERV_QPROC_TASKMSGFACTORY_H diff --git a/src/qproc/testQueryAnaGeneral.cc b/src/qproc/testQueryAnaGeneral.cc index cced49a07e..94dfcbadbe 100644 --- a/src/qproc/testQueryAnaGeneral.cc +++ b/src/qproc/testQueryAnaGeneral.cc @@ -745,9 +745,9 @@ BOOST_AUTO_TEST_CASE(SimpleScan) { BOOST_CHECK_EQUAL(context->dominantDb, std::string("LSST")); BOOST_CHECK(nullptr == context->secIdxRestrictors); BOOST_CHECK(nullptr == context->areaRestrictors); - BOOST_CHECK_EQUAL(context->scanInfo.infoTables.size(), 1U); - if (context->scanInfo.infoTables.size() >= 1) { - auto p = context->scanInfo.infoTables.front(); + BOOST_CHECK_EQUAL(context->scanInfo->infoTables.size(), 1U); + if (context->scanInfo->infoTables.size() >= 1) { + auto p = context->scanInfo->infoTables.front(); BOOST_CHECK_EQUAL(p.db, "LSST"); BOOST_CHECK_EQUAL(p.table, "Object"); } diff --git a/src/query/CMakeLists.txt b/src/query/CMakeLists.txt index 5df3c784c4..de326cb8fb 100644 --- a/src/query/CMakeLists.txt +++ b/src/query/CMakeLists.txt @@ -64,7 +64,6 @@ FUNCTION(query_tests) qmeta query rproc - xrdreq Boost::unit_test_framework Threads::Threads ) diff --git a/src/query/QueryContext.h b/src/query/QueryContext.h index a0a2ae942a..a263bafb95 100644 --- a/src/query/QueryContext.h +++ b/src/query/QueryContext.h @@ -38,7 +38,7 @@ // Local headers #include "css/CssAccess.h" #include "global/stringTypes.h" -#include "proto/ScanTableInfo.h" +#include "protojson/ScanTableInfo.h" #include "qana/QueryMapping.h" #include "query/FromList.h" #include "query/typedefs.h" @@ -83,7 +83,7 @@ class QueryContext { std::shared_ptr databaseModels; ///< contains database schema information. - proto::ScanInfo scanInfo; // Tables scanned (for shared scans) + protojson::ScanInfo::Ptr scanInfo{protojson::ScanInfo::create()}; // Tables scanned (for shared scans) /** * @brief Add a TableRef to the list of tables used by this query. diff --git a/src/query/QueryTemplate.cc b/src/query/QueryTemplate.cc index 699a6faab2..32e628e90d 100644 --- a/src/query/QueryTemplate.cc +++ b/src/query/QueryTemplate.cc @@ -43,6 +43,8 @@ #include "query/ColumnRef.h" #include "query/TableRef.h" +using namespace std; + namespace lsst::qserv::query { //////////////////////////////////////////////////////////////////////// @@ -204,4 +206,18 @@ QueryTemplate::GetAliasMode QueryTemplate::getTableAliasMode() const { return DONT_USE; // should never get here but to satisfy the compiler. } +string QueryTemplate::dump() const { + ostringstream os; + os << "QueryTemplate quoteIdents=" << _quoteIdentifiers; + os << " useColOnly=" << _useColumnOnly; + os << " aliasMode=" << _aliasMode; + os << " entries={"; + for (auto const& entry : _entries) { + os << "(dynamic=" << entry->isDynamic(); + os << ":val=" << entry->getValue() << ")"; + } + os << "}"; + return os.str(); +} + } // namespace lsst::qserv::query diff --git a/src/query/QueryTemplate.h b/src/query/QueryTemplate.h index 5be5e3ac03..b0ffad8ba2 100644 --- a/src/query/QueryTemplate.h +++ b/src/query/QueryTemplate.h @@ -208,6 +208,8 @@ class QueryTemplate { return os << qt.sqlFragment(); } + std::string dump() const; + private: EntryPtrVector _entries; SetAliasMode _aliasMode{USE_ALIAS}; diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index c02e7f7107..9eff6d90e8 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -172,6 +172,26 @@ bool InfileMerger::merge(proto::ResponseSummary const& resp, _setQueryIdStr(QueryIdHelper::makeIdStr(resp.queryid())); } + // Nothing to do if size is zero. + if (responseData.row_size() == 0) { + return true; + } + + // Do nothing if the query got cancelled for any reason. + if (jq->isQueryCancelled()) { + return true; + } + auto executive = jq->getExecutive(); + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { + return true; + } + + std::unique_ptr semaLock; + if (_dbEngine != MYISAM) { + // needed for parallel merging with INNODB and MEMORY + semaLock.reset(new util::SemaLock(*_semaMgrConn)); + } + TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, bool success) { if (!success) return; @@ -245,7 +265,7 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response return true; } auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { return true; } diff --git a/src/util/CMakeLists.txt b/src/util/CMakeLists.txt index c3213efb70..dc32c332b2 100644 --- a/src/util/CMakeLists.txt +++ b/src/util/CMakeLists.txt @@ -20,6 +20,7 @@ target_sources(util PRIVATE Issue.cc MultiError.cc Mutex.cc + QdispPool.cc ResultFileName.cc StringHash.cc String.cc @@ -30,7 +31,6 @@ target_sources(util PRIVATE Timer.cc TimeUtils.cc WorkQueue.cc - xrootd.cc ) target_link_libraries(util PUBLIC diff --git a/src/util/ConfigValMap.h b/src/util/ConfigValMap.h index 89b227ca79..610ef0bab5 100644 --- a/src/util/ConfigValMap.h +++ b/src/util/ConfigValMap.h @@ -50,6 +50,8 @@ class ConfigValMap; /// Base class for storing values, usually from configuration files, that have /// identifiers consisting of a `section` and a `name`. /// This class is meant to be used with ConfigValMap. +/// TODO:UJ a command line argument can be added to this and if the command +/// line argument is found, it will override the value in the file. class ConfigVal { public: using Ptr = std::shared_ptr; diff --git a/src/util/InstanceCount.cc b/src/util/InstanceCount.cc index af9f0f8dda..895698d63b 100644 --- a/src/util/InstanceCount.cc +++ b/src/util/InstanceCount.cc @@ -32,7 +32,7 @@ void InstanceCount::_increment(std::string const& source) { auto iter = ret.first; iter->second += 1; LOGS(_log, LOG_LVL_WARN, - "InstanceCount " << source << " " << iter->first << "=" << iter->second); // LockupDB INFO + "InstanceCount " << source << " " << iter->first << "=" << iter->second); //&&&DEBUG } InstanceCount::~InstanceCount() { @@ -41,7 +41,7 @@ InstanceCount::~InstanceCount() { if (iter != _instances.end()) { iter->second -= 1; LOGS(_log, LOG_LVL_WARN, - "~InstanceCount " << iter->first << "=" << iter->second << " : " << *this); // LockupDB INFO + "~InstanceCount " << iter->first << "=" << iter->second << " : " << *this); //&&&DEBUG if (iter->second == 0) { _instances.erase(_className); } diff --git a/src/util/Mutex.cc b/src/util/Mutex.cc index cd60e2b0e4..d7e46c0c13 100644 --- a/src/util/Mutex.cc +++ b/src/util/Mutex.cc @@ -47,7 +47,7 @@ void Lock::_lock() { _context << " LOCK[" << _mutex.id() << "]:1 " << " LOCKED: " << util::printable(Mutex::lockedId(), "", "", " ")); } - assert(!_mutex.lockedByCaller()); + assert(!_mutex.lockedByThread()); _mutex.lock(); if (!_context.empty()) { LOGS(_log, LOG_LVL_TRACE, diff --git a/src/util/Mutex.h b/src/util/Mutex.h index 0353f733a6..991db6b182 100644 --- a/src/util/Mutex.h +++ b/src/util/Mutex.h @@ -32,13 +32,33 @@ #include "util/Bug.h" +#define USING_VMUTEX 0 // &&& Should be replaced by variable in build. + +#ifdef MUTEX_UNITTEST +#define USING_VMUTEX 1 +#endif + +#if USING_VMUTEX + +#define MUTEX util::Mutex + /// Used to verify a mutex is locked before accessing a protected variable. #define VMUTEX_HELD(vmtx) \ - if (!vmtx.lockedByCaller()) throw lsst::qserv::util::Bug(ERR_LOC, "mutex not locked!"); + if (!vmtx.lockedByThread()) throw lsst::qserv::util::Bug(ERR_LOC, "mutex not locked!"); /// Used to verify a mutex is not locked by this thread before locking a related mutex. #define VMUTEX_NOT_HELD(vmtx) \ - if (vmtx.lockedByCaller()) throw lsst::qserv::util::Bug(ERR_LOC, "mutex not free!"); + if (vmtx.lockedByThread()) throw lsst::qserv::util::Bug(ERR_LOC, "mutex not unlocked!"); + +#else // not USING_VMUTEX + +#define MUTEX std::mutex + +#define VMUTEX_HELD(vmtx) ; + +#define VMUTEX_NOT_HELD(vmtx) ; + +#endif // USING_VMUTEX // This header declarations namespace lsst::qserv::util { @@ -50,6 +70,8 @@ namespace lsst::qserv::util { /// Making VMutex a wrapper around std::mutex instead of a child causes lines /// like `std::lock_guard lck(_vmutex);` to be flagged as errors, /// which is desirable. +/// Unfortunately, VMutex won't work with condition_variable as those explicitly +/// expect std::mutex. class VMutex { public: explicit VMutex() {} @@ -75,8 +97,7 @@ class VMutex { } /// @return true if the mutex is locked by this thread. - /// TODO: Rename lockedByThread() - bool lockedByCaller() const { return _holder == std::this_thread::get_id(); } + bool lockedByThread() const { return _holder == std::this_thread::get_id(); } protected: std::atomic _holder; @@ -101,13 +122,13 @@ class Mutex : public VMutex { Mutex() : _id(nextId()) {} - /// Lock the mutext (replaces the corresponding method of the base class) + /// Lock the mutex (replaces the corresponding method of the base class) void lock() { VMutex::lock(); addCurrentId(); } - /// Release the mutext (replaces the corresponding method of the base class) + /// Release the mutex (replaces the corresponding method of the base class) void unlock() { removeCurrentId(); VMutex::unlock(); diff --git a/src/qdisp/QdispPool.cc b/src/util/QdispPool.cc similarity index 96% rename from src/qdisp/QdispPool.cc rename to src/util/QdispPool.cc index 137e59a34b..02d2e1c41f 100644 --- a/src/qdisp/QdispPool.cc +++ b/src/util/QdispPool.cc @@ -22,7 +22,7 @@ */ // Class header -#include "qdisp/QdispPool.h" +#include "util/QdispPool.h" // LSST headers #include "lsst/log/Log.h" @@ -32,10 +32,10 @@ #include "util/common.h" namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.QdispPool"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.util.QdispPool"); } -namespace lsst::qserv::qdisp { +namespace lsst::qserv::util { ///< @Return true if the queue could be added. bool PriorityQueue::addPriQueue(int priority, int minRunning, int maxRunning) { @@ -67,6 +67,10 @@ void PriorityQueue::queCmd(util::Command::Ptr const& cmd) { void PriorityQueue::queCmd(PriorityCommand::Ptr const& cmd, int priority) { { std::lock_guard lock(_mtx); + if (cmd->_queued.exchange(true) == true) { + throw util::Bug(ERR_LOC, + "PriorityQueue::queCmd cmd has already been queued and cannot be queued twice."); + } auto iter = _queues.find(priority); if (iter == _queues.end()) { // give it the default priority @@ -162,6 +166,7 @@ void PriorityQueue::_incrDecrRunningCount(util::Command::Ptr const& cmd, int inc iter->second->running += incrDecr; } } + _cv.notify_one(); } void PriorityQueue::commandStart(util::Command::Ptr const& cmd) { @@ -260,4 +265,4 @@ QdispPool::QdispPool(bool unitTest) { } } -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::util diff --git a/src/qdisp/QdispPool.h b/src/util/QdispPool.h similarity index 91% rename from src/qdisp/QdispPool.h rename to src/util/QdispPool.h index d3e1af7743..562450624d 100644 --- a/src/qdisp/QdispPool.h +++ b/src/util/QdispPool.h @@ -20,8 +20,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_QDISP_QDISPPOOL_H -#define LSST_QSERV_QDISP_QDISPPOOL_H +#ifndef LSST_QSERV_UTIL_QDISPPOOL_H +#define LSST_QSERV_UTIL_QDISPPOOL_H // System headers #include @@ -33,7 +33,7 @@ // Qserv headers #include "util/ThreadPool.h" -namespace lsst::qserv::qdisp { +namespace lsst::qserv::util { class PriorityQueue; @@ -46,7 +46,10 @@ class PriorityCommand : public util::CommandTracked { friend PriorityQueue; private: - int _priority{0}; // Need to know what queue this was placed on. + int _priority{0}; ///< Need to know what queue this was placed on. + /// Priority commands can only be queued once, or PriorityQueue acounting + /// can be contaminated: this flag causes But to be thrown if queued twice. + std::atomic _queued{false}; }; /// FIFO priority queue. Elements with the same priority are handled in @@ -142,7 +145,7 @@ class PriorityQueue : public util::CommandQueue { /// This has not worked entirely as intended. Reducing the number of threads /// had negative impacts on xrootd, but other changes have been made such that /// reducing the size of the thread pools can be tried again. -/// What it does do is prioritize out going messages (typically jobs going to +/// What it does do is prioritize outgoing messages (typically jobs going to /// workers), allow interactive queries to be handled quickly, even under /// substantial loads, and it gives a good idea of how busy the czar really /// is. Large numbers of queued items in any of the scan queries, or large @@ -162,7 +165,8 @@ class QdispPool { /// largestPriority - highest priority is 0, lowest possible priority is /// 100 and is reserved for default priority. largestPriority=4 would /// result in PriorityQueues's being created for - /// priorities 0, 1, 2, 3, 4, and 100 + /// priorities 0, 1, 2, 3, 4, and 100. Priority 100 is + /// meant for changing aspects of the pool and shutdown. /// runSizes - Each entry represents the maximum number of concurrent running /// commands for a priority given by the position in the array. /// If a position is undefined, the default value is 1. @@ -172,7 +176,7 @@ class QdispPool { /// priorities 3 and 4 can have up to 3 /// minRunningSizes - Each entry represents the minimum number of threads /// to be running (defaults to 0). Non-zero values can keep - /// lower priorities from being completely stared and/or + /// lower priorities from being completely starved and/or /// reduce deadlocks from high priorities depending on lower /// priorities. QdispPool(int poolSize, int largestPriority, std::vector const& maxRunSizes, @@ -200,6 +204,6 @@ class QdispPool { util::ThreadPool::Ptr _pool; }; -} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::util -#endif /* LSST_QSERV_QDISP_QDISPPOOL_H_ */ +#endif /* LSST_QSERV_UTIL_QDISPPOOL_H_ */ diff --git a/src/util/testMutex.cc b/src/util/testMutex.cc index 42220436e6..e1da95c9d1 100644 --- a/src/util/testMutex.cc +++ b/src/util/testMutex.cc @@ -33,6 +33,8 @@ // LSST headers #include "lsst/log/Log.h" +#define MUTEX_UNITTEST + // Qserv headers #include "util/BlockPost.h" #include "util/Mutex.h" @@ -58,16 +60,16 @@ BOOST_AUTO_TEST_SUITE(Suite) BOOST_AUTO_TEST_CASE(MutexTest) { // Test the interface of class Mutex to comply with expectations // of the standard std::lock_guard. - LOGS_DEBUG("MutexTest begins"); + LOGS_INFO("MutexTest begins"); // The mutex won't be locked by anyone Mutex mtx1; - BOOST_CHECK(!mtx1.lockedByCaller()); + BOOST_CHECK(!mtx1.lockedByThread()); // The mutex will be locked by the current thread Mutex mtx2; lock_guard const lockGuard2(mtx2); - BOOST_CHECK(mtx2.lockedByCaller()); + BOOST_CHECK(mtx2.lockedByThread()); // Lock this mutex in each of two separate threads. Let each thread // to wait for a random period of time within some interval before @@ -85,18 +87,18 @@ BOOST_AUTO_TEST_CASE(MutexTest) { thread thr1([&mtx, &wasLockedBeforeBy1, &wasLockedAfterBy1]() { BlockPost blockPost(10, 20); blockPost.wait(); - wasLockedBeforeBy1 = mtx.lockedByCaller(); + wasLockedBeforeBy1 = mtx.lockedByThread(); lock_guard const lock(mtx); - wasLockedAfterBy1 = mtx.lockedByCaller(); + wasLockedAfterBy1 = mtx.lockedByThread(); }); bool wasLockedBeforeBy2 = false; bool wasLockedAfterBy2 = false; thread thr2([&mtx, &wasLockedBeforeBy2, &wasLockedAfterBy2]() { BlockPost blockPost(10, 20); blockPost.wait(); - wasLockedBeforeBy2 = mtx.lockedByCaller(); + wasLockedBeforeBy2 = mtx.lockedByThread(); lock_guard const lock(mtx); - wasLockedAfterBy2 = mtx.lockedByCaller(); + wasLockedAfterBy2 = mtx.lockedByThread(); }); thr1.join(); BOOST_CHECK(!wasLockedBeforeBy1); @@ -126,24 +128,24 @@ BOOST_AUTO_TEST_CASE(MutexTest) { } BOOST_CHECK_EQUAL(counter, steps * numThreads); } - LOGS_DEBUG("MutexTest ends"); + LOGS_INFO("MutexTest ends"); } BOOST_AUTO_TEST_CASE(VMutexTest) { // Test the interface of class Mutex to comply with expectations // of the standard std::lock_guard. - LOGS_DEBUG("VMutexTest begins"); + LOGS_INFO("VMutexTest begins"); // The mutex won't be locked by anyone VMutex mtx1; - BOOST_CHECK(!mtx1.lockedByCaller()); + BOOST_CHECK(!mtx1.lockedByThread()); BOOST_CHECK_THROW(VMUTEX_HELD(mtx1), lsst::qserv::util::Bug); BOOST_REQUIRE_NO_THROW(VMUTEX_NOT_HELD(mtx1)); // The mutex will be locked by the current thread VMutex mtx2; lock_guard const lockGuard2(mtx2); - BOOST_CHECK(mtx2.lockedByCaller()); + BOOST_CHECK(mtx2.lockedByThread()); BOOST_REQUIRE_NO_THROW(VMUTEX_HELD(mtx2)); BOOST_CHECK_THROW(VMUTEX_NOT_HELD(mtx2), lsst::qserv::util::Bug); @@ -163,18 +165,18 @@ BOOST_AUTO_TEST_CASE(VMutexTest) { thread thr1([&mtx, &wasLockedBeforeBy1, &wasLockedAfterBy1]() { BlockPost blockPost(10, 20); blockPost.wait(); - wasLockedBeforeBy1 = mtx.lockedByCaller(); + wasLockedBeforeBy1 = mtx.lockedByThread(); lock_guard const lock(mtx); - wasLockedAfterBy1 = mtx.lockedByCaller(); + wasLockedAfterBy1 = mtx.lockedByThread(); }); bool wasLockedBeforeBy2 = false; bool wasLockedAfterBy2 = false; thread thr2([&mtx, &wasLockedBeforeBy2, &wasLockedAfterBy2]() { BlockPost blockPost(10, 20); blockPost.wait(); - wasLockedBeforeBy2 = mtx.lockedByCaller(); + wasLockedBeforeBy2 = mtx.lockedByThread(); lock_guard const lock(mtx); - wasLockedAfterBy2 = mtx.lockedByCaller(); + wasLockedAfterBy2 = mtx.lockedByThread(); }); thr1.join(); BOOST_CHECK(!wasLockedBeforeBy1); @@ -205,16 +207,16 @@ BOOST_AUTO_TEST_CASE(VMutexTest) { BOOST_CHECK_EQUAL(counter, steps * numThreads); } - LOGS_DEBUG("VMutexTest ends"); + LOGS_INFO("VMutexTest ends"); } BOOST_AUTO_TEST_CASE(LockTest1) { // Test locking a mutex created on stack using a special class util::Lock. - LOGS_DEBUG("LockTest1 begins"); + LOGS_INFO("LockTest1 begins"); // The mutex won't be locked by anyone Mutex mtx1; - BOOST_CHECK(not mtx1.lockedByCaller()); + BOOST_CHECK(not mtx1.lockedByThread()); // The mutex will be locked by the current thread Mutex mtx2; @@ -222,9 +224,9 @@ BOOST_AUTO_TEST_CASE(LockTest1) { // Do this in a nested block to ensure that lock object // gets destructed before the mutex. Lock const lock(mtx2, "LockTes1t: main thread"); - BOOST_CHECK(mtx2.lockedByCaller()); + BOOST_CHECK(mtx2.lockedByThread()); } - LOGS_DEBUG(!mtx2.lockedByCaller()); + LOGS_INFO(!mtx2.lockedByThread()); // Lock this mutex in each of two separate threads. Let each thread // to wait for a random period of time within some interval before @@ -247,7 +249,7 @@ BOOST_AUTO_TEST_CASE(LockTest1) { blockPost.wait(); Lock const lock(mtx, "LockTest1: thread 2"); }); - BOOST_CHECK(!mtx.lockedByCaller()); + BOOST_CHECK(!mtx.lockedByThread()); thr1.join(); thr2.join(); } @@ -272,7 +274,7 @@ BOOST_AUTO_TEST_CASE(LockTest1) { } BOOST_CHECK_EQUAL(counter, steps * numThreads); } - LOGS_DEBUG("LockTest1 ends"); + LOGS_INFO("LockTest1 ends"); } BOOST_AUTO_TEST_CASE(LockTest2) { @@ -280,11 +282,11 @@ BOOST_AUTO_TEST_CASE(LockTest2) { // a shared pointer using a special class util::Lock. The test implements // the same testing algorithm as the previous test, except it will be testing // a different way of constructing the lock. - LOGS_DEBUG("LockTest2 begins"); + LOGS_INFO("LockTest2 begins"); // The mutex won't be locked by anyone shared_ptr const mtx1 = make_shared(); - BOOST_CHECK(!mtx1->lockedByCaller()); + BOOST_CHECK(!mtx1->lockedByThread()); // The mutex will be locked by the current thread shared_ptr const mtx2 = make_shared(); @@ -292,9 +294,9 @@ BOOST_AUTO_TEST_CASE(LockTest2) { // Do this in a nested block to ensure that lock object // gets destructed before the mutex. Lock const lock(mtx2, "LockTes1t: main thread"); - BOOST_CHECK(mtx2->lockedByCaller()); + BOOST_CHECK(mtx2->lockedByThread()); } - BOOST_CHECK(!mtx2->lockedByCaller()); + BOOST_CHECK(!mtx2->lockedByThread()); // Lock this mutex in each of two separate threads. Let each thread // to wait for a random period of time within some interval before @@ -317,7 +319,7 @@ BOOST_AUTO_TEST_CASE(LockTest2) { blockPost.wait(); Lock const lock(mtx, "LockTest1: thread 2"); }); - BOOST_CHECK(!mtx->lockedByCaller()); + BOOST_CHECK(!mtx->lockedByThread()); thr1.join(); thr2.join(); } @@ -342,7 +344,7 @@ BOOST_AUTO_TEST_CASE(LockTest2) { } BOOST_CHECK_EQUAL(counter, steps * numThreads); } - LOGS_DEBUG("LockTest2 ends"); + LOGS_INFO("LockTest2 ends"); } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/util/xrootd.cc b/src/util/xrootd.cc deleted file mode 100644 index a4f967faa4..0000000000 --- a/src/util/xrootd.cc +++ /dev/null @@ -1,89 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2009-2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -// xrootd.h -- Helper funcitons for xrootd-based dispatch - -#include "util/xrootd.h" - -// System headers -#include -#include - -// Third-party headers -#include "boost/format.hpp" - -namespace lsst::qserv::util { - -std::string makeUrl(char const* hostport, char const* typeStr, int chunk) { - std::stringstream s; - s << chunk; - // boost::format version is 5x slower. - // std::string s = (boost::format("%d") % chunk).str(); - return makeUrl(hostport, typeStr, s.str()); -} - -std::string makeUrl(char const* hostport, std::string const& path) { - return makeUrl(hostport, nullptr, path); -} - -std::string makeUrl(char const* hostport, char const* typeStr, std::string const& s, char mode) { - // typeStr is either "query" or "result" - if (!hostport) { - hostport = ::getenv("QSERV_XRD"); - if (!hostport) { - // use local host name if nothing is specified - hostport = "localhost:1094"; - } - } -#if 0 - char* user = "qsmaster"; - boost::format f("xroot://%s@%s//%s/%s"); - return (f % user % hostport % typeStr % s).str(); -#else - // This is ~8.5x faster than the boost::format version. - std::string pfx = "xroot://"; - std::string user("qsmaster"); - std::string tstr; - std::string ret; - if (typeStr) tstr = typeStr; - - if (mode != '\0') { - user += "."; - user += mode; - } - ret.reserve(pfx.size() + user.size() + 1 + 2 + 1 + tstr.size() + s.size()); - ret += pfx; - ret += user; - ret += "@"; - ret += hostport; - ret += "/"; - if (typeStr) { - ret += "/"; - ret += typeStr; - ret += "/"; - } // else: assume s contains leading "/" - ret += s; - return ret; -#endif -} - -} // namespace lsst::qserv::util diff --git a/src/util/xrootd.h b/src/util/xrootd.h deleted file mode 100644 index bf3c00f8bb..0000000000 --- a/src/util/xrootd.h +++ /dev/null @@ -1,42 +0,0 @@ -// -*- LSST-C++ -*- - -/* - * LSST Data Management System - * Copyright 2008, 2009, 2010 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_UTIL_XROOTD_H -#define LSST_QSERV_UTIL_XROOTD_H - -// xrootd.h : consolidates xrootd/lower-level helper functions (i.e., -// dealing with xrootd URLs) - -// Third-party headers -#include - -namespace lsst::qserv::util { - -std::string makeUrl(char const* hostport, char const* typeStr, int chunk); -std::string makeUrl(char const* hostport, char const* typeStr, std::string const& s, char mode = 0); -std::string makeUrl(char const* hostport, std::string const& path); - -} // namespace lsst::qserv::util - -#endif // LSST_QSERV_UTIL_XROOTD_H diff --git a/src/wbase/CMakeLists.txt b/src/wbase/CMakeLists.txt index 7f0365cb73..33ebbef0a0 100644 --- a/src/wbase/CMakeLists.txt +++ b/src/wbase/CMakeLists.txt @@ -8,7 +8,6 @@ target_sources(wbase PRIVATE Task.cc UberJobData.cc UserQueryInfo.cc - WorkerCommand.cc ) install( diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index 9ad5f60eed..38eaf9ab57 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -47,7 +47,6 @@ #include "util/ResultFileName.h" #include "util/Timer.h" #include "util/TimeUtils.h" -#include "xrdsvc/StreamBuffer.h" // LSST headers #include "lsst/log/Log.h" @@ -298,8 +297,7 @@ FileChannelShared::FileChannelShared(std::shared_ptr const& _czarPort(czarPort), _workerId(workerId), _protobufArena(make_unique()), - _scsId(scsSeqId++), - _useHttp(true) { + _scsId(scsSeqId++) { LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created scsId=" << _scsId << " ujId=" << _uberJobId); } @@ -309,25 +307,34 @@ FileChannelShared::~FileChannelShared() { // dead it means there was a problem to process a query or send back a response // to Czar. In either case, the file would be useless and it has to be deleted // in order to avoid leaving unclaimed result files within the results folder. - if (isDead()) { + // + // _rowLimitComplete confuses things as it can cause other Tasks using this + // file to be cancelled, but the file should not be deleted until collected. + // In any case, the WorkerQueryStatusData message from the czar will delete + // the file when the user query completes. + if (isDead() && !_rowLimitComplete) { _removeFile(lock_guard(_tMtx)); } - if (!_useHttp) { - if (_sendChannel != nullptr) { - _sendChannel->setDestroying(); - if (!_sendChannel->isDead()) { - _sendChannel->kill("~FileChannelShared()"); - } - } - } LOGS(_log, LOG_LVL_DEBUG, "~FileChannelShared end"); } void FileChannelShared::setTaskCount(int taskCount) { _taskCount = taskCount; } -bool FileChannelShared::transmitTaskLast() { +bool FileChannelShared::transmitTaskLast(bool rowLimitComplete) { lock_guard const streamMutexLock(_streamMutex); ++_lastCount; + if (rowLimitComplete) { + // There are enough rows in the file so other tasks can be ignored. + if (_rowLimitComplete.exchange(true) == false) { + // This is TaskLast. + return true; + } else { + // A different task set _rowLimitComplete before + // this one. Since there can be only one TaskLast, + // it is not this one. + return false; + } + } bool lastTaskDone = _lastCount >= _taskCount; return lastTaskDone; } @@ -337,41 +344,29 @@ bool FileChannelShared::kill(string const& note) { return _kill(streamMutexLock, note); } -bool FileChannelShared::isDead() { - if (!_useHttp) { - if (_sendChannel == nullptr) return true; - return _sendChannel->isDead(); - } else { - return _dead; - } -} +bool FileChannelShared::isDead() const { return _dead; } string FileChannelShared::makeIdStr(int qId, int jId) { string str("QID" + (qId == 0 ? "" : to_string(qId) + "#" + to_string(jId))); return str; } +bool FileChannelShared::isRowLimitComplete() const { + lock_guard const tMtxLock(_tMtx); + return _rowLimitComplete; +} + bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared_ptr const& task, bool cancelled) { lock_guard const tMtxLock(_tMtx); - if (!_useHttp) { - if (!_sendResponse(tMtxLock, task, cancelled, multiErr)) { - LOGS(_log, LOG_LVL_ERROR, "Could not transmit the error message to Czar."); - return false; - } - return true; - } else { - auto ujData = _uberJobData.lock(); - if (ujData == nullptr) { - LOGS(_log, LOG_LVL_WARN, - __func__ << " not sending error as ujData is null " << multiErr.toString()); - return false; - } - // Delete the result file as nobody will come looking for it. - _kill(tMtxLock, " buildAndTransmitError"); - return ujData->responseError(multiErr, task, cancelled); + if (_rowLimitComplete) { + LOGS(_log, LOG_LVL_WARN, + __func__ << " already enough rows, this call likely a side effect" << task->getIdStr()); + return false; } - return false; + // Delete the result file as nobody will come looking for it. + _kill(tMtxLock, " buildAndTransmitError"); + return _uberJobData->responseError(multiErr, task, cancelled); } bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& task, @@ -396,6 +391,11 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const tMtxLockA(_tMtx); + if (_rowLimitComplete) { + LOGS(_log, LOG_LVL_DEBUG, __func__ << " already enough rows, returning " << task->getIdStr()); + // Deleting the file now could be risky. + return erred; + } // Extract the result set and write it into the file. util::Timer bufferFillT; @@ -411,12 +411,9 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetMaxTableSize(); - if (maxTableSize > 0 && _transmitsize > maxTableSize) { - string const err = "The result set size " + to_string(_transmitsize) + + // "large result" limit (in case one was specified). + if (maxTableSize > 0 && bytesTransmitted > maxTableSize) { + string const err = "The result set size " + to_string(bytesTransmitted) + " of a job exceeds the requested limit of " + to_string(maxTableSize) + " bytes, task: " + task->getIdStr(); multiErr.push_back(util::Error(util::ErrorCode::WORKER_RESULT_TOO_LARGE, err)); @@ -424,17 +421,28 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetRowLimit(); + bool rowLimitComplete = false; + if (ujRowLimit > 0 && _rowcount >= ujRowLimit) { + // There are enough rows to satisfy the query, so stop reading + hasMoreRows = false; + rowLimitComplete = true; + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " enough rows for query rows=" << _rowcount << " " << task->getIdStr()); + } + + // If no more rows are left in the task's result set then we need to check + // if this is last task in a logical group of ones created for processing // the current request (note that certain classes of requests may require // more than one task for processing). - if (!erred && transmitTaskLast()) { + if (!hasMoreRows && transmitTaskLast(rowLimitComplete)) { // Make sure the file is sync to disk before notifying Czar. _file.flush(); _file.close(); // Only the last ("summary") message, w/o any rows, is sent to the Czar to notify // it about the completion of the request. - if (!_sendResponse(tMtxLockA, task, cancelled, multiErr)) { + if (!_sendResponse(tMtxLockA, task, cancelled, multiErr, rowLimitComplete)) { LOGS(_log, LOG_LVL_ERROR, "Could not transmit the request completion message to Czar."); erred = true; } else { @@ -458,7 +466,7 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const tMtxLockA(_tMtx); _removeFile(tMtxLockA); } @@ -467,16 +475,11 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& streamMutexLock, string const& note) { LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared::" << __func__ << " " << note); - if (!_useHttp) { - return _sendChannel->kill(note); - } else { - bool oldVal = _dead.exchange(true); - if (!oldVal) { - LOGS(_log, LOG_LVL_WARN, "FileChannelShared first kill call " << note); - } - _removeFile(streamMutexLock); - return oldVal; + bool oldVal = _dead.exchange(true); + if (!oldVal) { + LOGS(_log, LOG_LVL_WARN, "FileChannelShared first kill call " << note); } + return oldVal; } void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_ptr const& task, @@ -543,7 +546,7 @@ void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { } bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ptr const& task, - bool cancelled, util::MultiError const& multiErr) { + bool cancelled, util::MultiError const& multiErr, bool mustSend) { auto const queryId = task->getQueryId(); auto const jobId = task->getJobId(); auto const idStr(makeIdStr(queryId, jobId)); @@ -554,99 +557,15 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ QSERV_LOGCONTEXT_QUERY_JOB(queryId, jobId); LOGS(_log, LOG_LVL_DEBUG, __func__); - if (isDead()) { + if (isDead() && !mustSend) { LOGS(_log, LOG_LVL_INFO, __func__ << ": aborting transmit since sendChannel is dead."); return false; } // Prepare the response object and serialize in into a message that will - // be sent to Czar. - proto::ResponseSummary response; - response.set_wname(_workerId); - response.set_queryid(queryId); - response.set_jobid(jobId); - response.set_fileresource_http(task->resultFileHttpUrl()); - response.set_attemptcount(task->getAttemptCount()); - response.set_rowcount(_rowcount); - response.set_transmitsize(_transmitsize); - string errorMsg; - int errorCode = 0; - if (!multiErr.empty()) { - errorMsg = multiErr.toOneLineString(); - errorCode = multiErr.firstErrorCode(); - } else if (cancelled) { - errorMsg = "cancelled"; - errorCode = -1; - } - if (!errorMsg.empty() or (errorCode != 0)) { - errorMsg = "FileChannelShared::" + string(__func__) + " error(s) in result for chunk #" + - to_string(task->getChunkId()) + ": " + errorMsg; - response.set_errormsg(errorMsg); - response.set_errorcode(errorCode); - LOGS(_log, LOG_LVL_ERROR, errorMsg); - } - response.SerializeToString(&_responseBuf); - - LOGS(_log, LOG_LVL_DEBUG, - __func__ << " idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size()); - - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&& idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size() << " useHttp=" << _useHttp); - if (!_useHttp) { - proto::ResponseSummary response; - response.set_wname(_workerId); - response.set_queryid(queryId); - response.set_jobid(jobId); - response.set_fileresource_xroot(task->resultFileXrootUrl()); - response.set_fileresource_http(task->resultFileHttpUrl()); - response.set_attemptcount(task->getAttemptCount()); - response.set_rowcount(_rowcount); - response.set_transmitsize(_transmitsize); - string errorMsg; - int errorCode = 0; - if (!multiErr.empty()) { - errorMsg = multiErr.toOneLineString(); - errorCode = multiErr.firstErrorCode(); - } else if (cancelled) { - errorMsg = "cancelled"; - errorCode = -1; - } - if (!errorMsg.empty() or (errorCode != 0)) { - errorMsg = "FileChannelShared::" + string(__func__) + " error(s) in result for chunk #" + - to_string(task->getChunkId()) + ": " + errorMsg; - response.set_errormsg(errorMsg); - response.set_errorcode(errorCode); - LOGS(_log, LOG_LVL_ERROR, errorMsg); - } - response.SerializeToString(&_responseBuf); - - LOGS(_log, LOG_LVL_DEBUG, - __func__ << " idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size()); - - // Send the message sent out-of-band within the SSI metadata. - if (!_sendChannel->setMetadata(_responseBuf.data(), _responseBuf.size())) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed in setMetadata " << idStr); - _kill(streamMutexLock, "setMetadata"); - return false; - } - - // Send back the empty object since no info is expected by a caller - // for this type of requests beyond the usual error notifications (if any). - // Note that this call is needed to initiate the transaction. - if (!_sendChannel->sendData((char const*)0, 0)) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed in sendData " << idStr); - _kill(streamMutexLock, "sendData"); - return false; - } - } else { - auto ujData = _uberJobData.lock(); - if (ujData == nullptr) { - LOGS(_log, LOG_LVL_WARN, __func__ << " uberJobData is nullptr for ujId=" << _uberJobId); - return false; - } - string httpFileUrl = task->resultFileHttpUrl(); - ujData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); - } + // be sent to the Czar. + string httpFileUrl = task->resultFileHttpUrl(); + _uberJobData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); return true; } diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 1e23d5b8d0..649bbae374 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -129,7 +129,9 @@ class FileChannelShared { int getTaskCount() const { return _taskCount; } /// @return true if this is the last task to call this - bool transmitTaskLast(); + /// @param rowLimitComplete - true means enough rows for the result are + /// already in the file, so other tasks can be ignored. + bool transmitTaskLast(bool rowLimitComplete); /// Return a normalized id string. static std::string makeIdStr(int qId, int jId); @@ -156,7 +158,12 @@ class FileChannelShared { bool kill(std::string const& note); /// @see wbase::SendChannel::isDead - bool isDead(); + bool isDead() const; + + /// Return true if there are enough rows in this result file to satisfy the + /// LIMIT portion of the query. + /// @See _rowLimitComplete + bool isRowLimitComplete() const; private: /// TODO:UJ delete sendchannel version of constructor when possible. @@ -218,17 +225,18 @@ class FileChannelShared { * @param task - a task that produced the result set * @param cancelled - request cancellaton flag (if any) * @param multiErr - a collector of any errors that were captured during result set processing + * @param mustSend - set to true if this message should be sent even if the query was cancelled. * @return 'true' if the operation was successfull */ bool _sendResponse(std::lock_guard const& tMtxLock, std::shared_ptr const& task, - bool cancelled, util::MultiError const& multiErr); + bool cancelled, util::MultiError const& multiErr, bool mustSend = false); mutable std::mutex _tMtx; ///< Protects data recording and Czar notification bool _isUberJob; ///< true if this is using UberJob http. To be removed when _sendChannel goes away. std::shared_ptr const _sendChannel; ///< Used to pass encoded information to XrdSsi. - std::weak_ptr _uberJobData; ///< Pointer to UberJobData + std::shared_ptr _uberJobData; ///< Contains czar contact info. UberJobId const _uberJobId; ///< The UberJobId qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). TODO:UJ delete @@ -266,11 +274,15 @@ class FileChannelShared { // Counters reported to Czar in the only ("summary") message sent upon the completion // of all tasks of a query. - uint32_t _rowcount = 0; ///< The total numnber of rows in all result sets of a query. + int64_t _rowcount = 0; ///< The total numnber of rows in all result sets of a query. uint64_t _transmitsize = 0; ///< The total amount of data (bytes) in all result sets of a query. uint64_t _headerCount = 0; ///< Count of headers received. - bool const _useHttp = false; ///< to be eliminated when xrootd is no longer used. + /// _rowLimitComplete indicates that there is a LIMIT clause in the user query that + /// can be applied to the queries given to workers. It's important to apply it + /// when possible as an UberJob could have 1000 chunks and a LIMIT of 1, and it's + /// much faster to answer the query without scanning all 1000 chunks. + std::atomic _rowLimitComplete; std::atomic _dead{false}; ///< Set to true when the contents of the file are no longer useful. }; diff --git a/src/wbase/MsgProcessor.h b/src/wbase/MsgProcessor.h deleted file mode 100644 index 8458dc3f45..0000000000 --- a/src/wbase/MsgProcessor.h +++ /dev/null @@ -1,64 +0,0 @@ - -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2011-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// MsgProcessor.h -#ifndef LSST_QSERV_WBASE_MSG_PROCESSOR_H -#define LSST_QSERV_WBASE_MSG_PROCESSOR_H - -// System headers -#include -#include - -// Third party headers -#include "nlohmann/json.hpp" - -// Forward declarations -namespace lsst::qserv::wbase { -class Task; -struct TaskSelector; -class WorkerCommand; -} // namespace lsst::qserv::wbase - -namespace lsst::qserv::wbase { - -/// MsgProcessor implementations handle incoming Task objects. -struct MsgProcessor { - virtual ~MsgProcessor() {} - - /// Process a group of query processing tasks. - virtual void processTasks(std::vector> const& tasks) = 0; - - /// Process a managememt command - virtual void processCommand(std::shared_ptr const& command) = 0; - - /** - * Retreive the status of queries being processed by the worker. - * @param taskSelector Task selection criterias. - * @return a JSON representation of the object's status for the monitoring - */ - virtual nlohmann::json statusToJson(wbase::TaskSelector const& taskSelector) = 0; -}; - -} // namespace lsst::qserv::wbase - -#endif // LSST_QSERV_WBASE_MSG_PROCESSOR_H diff --git a/src/wbase/SendChannel.cc b/src/wbase/SendChannel.cc index 21e459ee87..c07dd37f63 100644 --- a/src/wbase/SendChannel.cc +++ b/src/wbase/SendChannel.cc @@ -41,7 +41,6 @@ #include "global/LogContext.h" #include "util/common.h" #include "util/Timer.h" -#include "xrdsvc/SsiRequest.h" namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.SendChannel"); @@ -61,19 +60,9 @@ class NopChannel : public SendChannel { cout << "NopChannel send(" << (void*)buf << ", " << bufLen << ");\n"; return !isDead(); } - - bool sendError(string const& msg, int code) override { - if (kill("NopChannel")) return false; - cout << "NopChannel sendError(\"" << msg << "\", " << code << ");\n"; - return true; - } - bool sendStream(xrdsvc::StreamBuffer::Ptr const& sBuf, bool last) override { - cout << "NopChannel sendStream(" << (void*)sBuf.get() << ", " << (last ? "true" : "false") << ");\n"; - return !isDead(); - } }; -SendChannel::Ptr SendChannel::newNopChannel() { return make_shared(); } +SendChannel::Ptr SendChannel::newNopChannel() { return std::shared_ptr(new NopChannel()); } /// StringChannel is an almost-trivial implementation of a SendChannel that /// remembers what it has received. @@ -87,46 +76,12 @@ class StringChannel : public SendChannel { return true; } - bool sendError(string const& msg, int code) override { - if (kill("StringChannel")) return false; - ostringstream os; - os << "(" << code << "," << msg << ")"; - _dest.append(os.str()); - return true; - } - - bool sendStream(xrdsvc::StreamBuffer::Ptr const& sBuf, bool last) override { - if (isDead()) return false; - char const* buf = sBuf->data; - size_t bufLen = sBuf->getSize(); - _dest.append(buf, bufLen); - cout << "StringChannel sendStream(" << (void*)buf << ", " << bufLen << ", " - << (last ? "true" : "false") << ");\n"; - return true; - } - private: string& _dest; }; -SendChannel::Ptr SendChannel::newStringChannel(string& d) { return make_shared(d); } - -/// This is the standard definition of SendChannel which actually does something! -/// We vector responses posted to SendChannel via the tightly bound SsiRequest -/// object as this object knows how to effect Ssi responses. -/// -bool SendChannel::send(char const* buf, int bufLen) { - if (isDead()) return false; - if (_ssiRequest->reply(buf, bufLen)) return true; - kill("SendChannel::send"); - return false; -} - -bool SendChannel::sendError(string const& msg, int code) { - // Kill this send channel. If it wasn't already dead, send the error. - if (kill("SendChannel::sendError")) return false; - if (_ssiRequest->replyError(msg.c_str(), code)) return true; - return false; +SendChannel::Ptr SendChannel::newStringChannel(string& d) { + return std::shared_ptr(new StringChannel(d)); } bool SendChannel::kill(std::string const& note) { @@ -139,36 +94,7 @@ bool SendChannel::kill(std::string const& note) { bool SendChannel::isDead() { if (_dead) return true; - if (_ssiRequest == nullptr) return true; - if (_ssiRequest->isFinished()) kill("SendChannel::isDead"); return _dead; } -bool SendChannel::sendStream(xrdsvc::StreamBuffer::Ptr const& sBuf, bool last) { - if (isDead()) return false; - if (_ssiRequest->replyStream(sBuf, last)) return true; - LOGS(_log, LOG_LVL_ERROR, "_ssiRequest->replyStream failed, killing."); - kill("SendChannel::sendStream"); - return false; -} - -bool SendChannel::sendData(char const* buf, int bufLen) { - if (isDead()) return false; - if (_ssiRequest->reply(buf, bufLen)) return true; - LOGS(_log, LOG_LVL_ERROR, "_ssiRequest->reply failed, killing."); - kill("SendChannel::sendData"); - return false; -} - -bool SendChannel::setMetadata(const char* buf, int blen) { - if (isDead()) return false; - if (_ssiRequest->sendMetadata(buf, blen)) return true; - return false; -} - -uint64_t SendChannel::getSeq() const { - if (_ssiRequest == nullptr) return 0; - return _ssiRequest->getSeq(); -} - } // namespace lsst::qserv::wbase diff --git a/src/wbase/SendChannel.h b/src/wbase/SendChannel.h index 0753e0aeff..56f2a598c8 100644 --- a/src/wbase/SendChannel.h +++ b/src/wbase/SendChannel.h @@ -23,18 +23,12 @@ #define LSST_QSERV_WBASE_SENDCHANNEL_H // System headers +#include #include #include #include -// Qserv headers -#include "xrdsvc/StreamBuffer.h" - -namespace lsst::qserv { -namespace xrdsvc { -class SsiRequest; // Forward declaration -} -namespace wbase { +namespace lsst::qserv { namespace wbase { /// SendChannel objects abstract an byte-output mechanism. Provides a layer of /// abstraction to reduce coupling to the XrdSsi API. SendChannel generally @@ -44,35 +38,13 @@ class SendChannel { using Ptr = std::shared_ptr; using Size = long long; - SendChannel(std::shared_ptr const& s) : _ssiRequest(s) {} SendChannel() {} // Strictly for non-Request versions of this object. virtual ~SendChannel() {} - /// ****************************************************************** /// The following methods are used to send responses back to a request. - /// The "send" calls may vector the response via the tightly bound - /// SsiRequest object (the constructor default) or use some other - /// mechanism (see newNopChannel and newStringChannel). - /// - virtual bool send(char const* buf, int bufLen); - virtual bool sendError(std::string const& msg, int code); - - /// Send a bucket of bytes. - /// @param last true if no more sendStream calls will be invoked. - virtual bool sendStream(xrdsvc::StreamBuffer::Ptr const& sBuf, bool last); - - /// Send the data. - virtual bool sendData(char const* buf, int bufLen); - - /// - /// ****************************************************************** - - /// Set a function to be called when a resources from a deferred send* - /// operation may be released. This allows a caller to be - /// notified when the file descriptor may be closed and perhaps reclaimed. - void setReleaseFunc(std::function const& r) { _release = r; } - void release() { _release(); } + /// (see newNopChannel and newStringChannel). + virtual bool send(char const* buf, int bufLen) = 0; // TODO:UJ remove + change unit tests /// Construct a new NopChannel that ignores everything it is asked to send static SendChannel::Ptr newNopChannel(); @@ -81,10 +53,6 @@ class SendChannel { /// provided by reference at construction. static SendChannel::Ptr newStringChannel(std::string& dest); - /// @return true if metadata was set. - /// buff must remain valid until the transmit is complete. - bool setMetadata(const char* buf, int blen); - /// Kill this SendChannel /// @ return the previous value of _dead bool kill(std::string const& note); @@ -95,17 +63,10 @@ class SendChannel { /// Set just before destorying this object to prevent pointless error messages. void setDestroying() { _destroying = true; } - uint64_t getSeq() const; - -protected: - std::function _release = []() { ; }; ///< Function to release resources. - private: - std::shared_ptr _ssiRequest; std::atomic _dead{false}; ///< True if there were any failures using this SendChanel. std::atomic _destroying{false}; }; -} // namespace wbase -} // namespace lsst::qserv +}} // namespace lsst::qserv::wbase #endif // LSST_QSERV_WBASE_SENDCHANNEL_H diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 4d276db677..55e29043f3 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -46,9 +46,10 @@ #include "global/constants.h" #include "global/LogContext.h" #include "global/UnsupportedError.h" -#include "http/RequestBody.h" +#include "http/RequestBodyJSON.h" #include "mysql/MySqlConfig.h" #include "proto/worker.pb.h" +#include "protojson/UberJobMsg.h" #include "util/Bug.h" #include "util/common.h" #include "util/HoldTrack.h" @@ -72,18 +73,6 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.Task"); -string buildResultFileName(shared_ptr const& taskMsg) { - auto const resultFileName = - lsst::qserv::util::ResultFileName(taskMsg->czarid(), taskMsg->queryid(), taskMsg->jobid(), - taskMsg->chunkid(), taskMsg->attemptcount()); - return resultFileName.fileName(); -} - -string buildResultFilePath(string const& resultFileName, string const& resultsDirname) { - if (resultsDirname.empty()) return resultsDirname; - return fs::weakly_canonical(fs::path(resultsDirname) / resultFileName).string(); -} - string buildUjResultFilePath(lsst::qserv::wbase::UberJobData::Ptr const& ujData, string const& resultsDirname) { if (resultsDirname.empty()) return resultsDirname; @@ -131,103 +120,17 @@ TaskScheduler::TaskScheduler() { atomic taskSequence{0}; ///< Unique identifier source for Task. -/// When the constructor is called, there is not enough information -/// available to define the action to take when this task is run, so -/// Command::setFunc() is used set the action later. This is why -/// the util::CommandThreadPool is not called here. -Task::Task(TaskMsgPtr const& t, int fragmentNumber, shared_ptr const& userQueryInfo, - size_t templateId, int subchunkId, shared_ptr const& sc, - uint16_t resultsHttpPort) - : _userQueryInfo(userQueryInfo), - _sendChannel(sc), - _tSeq(++taskSequence), - _qId(t->queryid()), - _templateId(templateId), - _hasChunkId(t->has_chunkid()), - _chunkId(t->has_chunkid() ? t->chunkid() : -1), - _subchunkId(subchunkId), - _jId(t->jobid()), - _attemptCount(t->attemptcount()), - _queryFragmentNum(fragmentNumber), - _fragmentHasSubchunks(t->fragment(fragmentNumber).has_subchunks()), - _db(t->has_db() ? t->db() : ""), - _czarId(t->has_czarid() ? t->czarid() : -1) { - // These attributes will be passed back to Czar in the Protobuf response - // to advice which result delivery channel to use. - auto const workerConfig = wconfig::WorkerConfig::instance(); - _resultFileName = ::buildResultFileName(t); - _resultFileAbsPath = ::buildResultFilePath(_resultFileName, workerConfig->resultsDirname()); - _resultFileHttpUrl = "http://" + _fqdn + ":" + to_string(resultsHttpPort) + "/" + _resultFileName; - if (t->has_user()) { - user = t->user(); - } else { - user = defaultUser; - } - - // Determine which major tables this task will use. - int const size = t->scantable_size(); - for (int j = 0; j < size; ++j) { - _scanInfo.infoTables.push_back(proto::ScanTableInfo(t->scantable(j))); - } - _scanInfo.scanRating = t->scanpriority(); - _scanInfo.sortTablesSlowestFirst(); - _scanInteractive = t->scaninteractive(); - _maxTableSize = t->maxtablesize_mb() * ::MB_SIZE_BYTES; - - // Create sets and vectors for 'aquiring' subchunk temporary tables. - proto::TaskMsg_Fragment const& fragment(t->fragment(_queryFragmentNum)); - DbTableSet dbTbls_; - IntVector subchunksVect_; - if (!_fragmentHasSubchunks) { - /// FUTURE: Why acquire anything if there are no subchunks in the fragment? - /// This branch never seems to happen, but this needs to be proven beyond any doubt. - LOGS(_log, LOG_LVL_WARN, "Task::Task not _fragmentHasSubchunks"); - for (auto const& scanTbl : t->scantable()) { - dbTbls_.emplace(scanTbl.db(), scanTbl.table()); - LOGS(_log, LOG_LVL_INFO, - "Task::Task scanTbl.db()=" << scanTbl.db() << " scanTbl.table()=" << scanTbl.table()); - } - LOGS(_log, LOG_LVL_INFO, - "fragment a db=" << _db << ":" << _chunkId << " dbTbls=" << util::printable(dbTbls_)); - } else { - proto::TaskMsg_Subchunk const& sc = fragment.subchunks(); - for (int j = 0; j < sc.dbtbl_size(); j++) { - /// Different subchunk fragments can require different tables. - /// FUTURE: It may save space to store these in UserQueryInfo as it seems - /// database and table names are consistent across chunks. - dbTbls_.emplace(sc.dbtbl(j).db(), sc.dbtbl(j).tbl()); - LOGS(_log, LOG_LVL_TRACE, - "Task::Task subchunk j=" << j << " sc.dbtbl(j).db()=" << sc.dbtbl(j).db() - << " sc.dbtbl(j).tbl()=" << sc.dbtbl(j).tbl()); - } - IntVector sVect(sc.id().begin(), sc.id().end()); - subchunksVect_ = sVect; - if (sc.has_database()) { - _db = sc.database(); - } else { - _db = t->db(); - } - LOGS(_log, LOG_LVL_DEBUG, - "fragment b db=" << _db << ":" << _chunkId << " dbTableSet" << util::printable(dbTbls_) - << " subChunks=" << util::printable(subchunksVect_)); - } - _dbTblsAndSubchunks = make_unique(dbTbls_, subchunksVect_); - if (_sendChannel == nullptr) { - throw util::Bug(ERR_LOC, "Task::Task _sendChannel==null " + getIdStr()); - } -} - /// When the constructor is called, there is not enough information /// available to define the action to take when this task is run, so /// Command::setFunc() is used set the action later. This is why /// the util::CommandThreadPool is not called here. Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, - shared_ptr const& userQueryInfo, size_t templateId, bool hasSubchunks, - int subchunkId, string const& db, proto::ScanInfo const& scanInfo, bool scanInteractive, - int maxTableSize, vector const& fragSubTables, vector const& fragSubchunkIds, - shared_ptr const& sc, uint16_t resultsHttpPort) - : _userQueryInfo(userQueryInfo), - _sendChannel(sc), + size_t templateId, bool hasSubchunks, int subchunkId, string const& db, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSize, + vector const& fragSubTables, vector const& fragSubchunkIds, + shared_ptr const& sc, + std::shared_ptr const& queryStats_, uint16_t resultsHttpPort) + : _sendChannel(sc), _tSeq(++taskSequence), _qId(ujData->getQueryId()), _templateId(templateId), @@ -242,7 +145,9 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun _czarId(ujData->getCzarId()), _scanInfo(scanInfo), _scanInteractive(scanInteractive), - _maxTableSize(maxTableSize * ::MB_SIZE_BYTES) { + _queryStats(queryStats_), + _maxTableSize(maxTableSize * ::MB_SIZE_BYTES), + _rowLimit(ujData->getRowLimit()) { // These attributes will be passed back to Czar in the Protobuf response // to advice which result delivery channel to use. auto const workerConfig = wconfig::WorkerConfig::instance(); @@ -265,7 +170,7 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun if (!_fragmentHasSubchunks) { /// FUTURE: Why acquire anything if there are no subchunks in the fragment? /// This branch never seems to happen, but this needs to be proven beyond any doubt. - for (auto const& scanTbl : scanInfo.infoTables) { + for (auto const& scanTbl : scanInfo->infoTables) { dbTbls_.emplace(scanTbl.db, scanTbl.table); LOGS(_log, LOG_LVL_INFO, "Task::Task scanTbl.db=" << scanTbl.db << " scanTbl.table=" << scanTbl.table); @@ -291,81 +196,234 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun _dbTblsAndSubchunks = make_unique(dbTbls_, subchunksVect_); } -Task::~Task() { - _userQueryInfo.reset(); - UserQueryInfo::uqMapErase(_qId); - if (UserQueryInfo::uqMapGet(_qId) == nullptr) { - LOGS(_log, LOG_LVL_TRACE, "~Task Cleared uqMap entry for _qId=" << _qId); - } -} +Task::~Task() {} + +/* &&& +std::vector Task::createTasksForChunk( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, + std::shared_ptr const& sqlConnMgr, + std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { + QueryId qId = ujData->getQueryId(); + UberJobId ujId = ujData->getUberJobId(); + CzarIdType czId = ujData->getCzarId(); + + wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); + UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); + + string funcN(__func__); + funcN += " QID=" + to_string(qId) + " "; -vector Task::createTasks(shared_ptr const& taskMsg, - shared_ptr const& sendChannel, - shared_ptr const& chunkResourceMgr, - mysql::MySqlConfig const& mySqlConfig, - shared_ptr const& sqlConnMgr, - shared_ptr const& queriesAndChunks, - uint16_t resultsHttpPort) { - QueryId qId = taskMsg->queryid(); - QSERV_LOGCONTEXT_QUERY_JOB(qId, taskMsg->jobid()); vector vect; + for (auto const& job : jsJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBodyJSON rbJobDesc(jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + auto const jdCzarId = rbJobDesc.required("czarId"); + auto const jdQueryId = rbJobDesc.required("queryId"); + if (jdQueryId != qId) { + throw TaskException(ERR_LOC, string("ujId=") + to_string(ujId) + " qId=" + to_string(qId) + + " QueryId mismatch Job qId=" + to_string(jdQueryId)); + } + auto const jdJobId = rbJobDesc.required("jobId"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_TRACE, + funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); + + auto const jdQueryFragments = rbJobDesc.required("queryFragments"); + int fragmentNumber = 0; + for (auto const& frag : jdQueryFragments) { + vector fragSubQueries; + vector fragSubchunkIds; + vector fragSubTables; + LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); + http::RequestBodyJSON rbFrag(frag); + auto const& jsQueries = rbFrag.required("queries"); + // TODO:UJ move to uberjob???, these should be the same for all jobs + for (auto const& subQ : jsQueries) { + http::RequestBodyJSON rbSubQ(subQ); + auto const subQuery = rbSubQ.required("subQuery"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); + fragSubQueries.push_back(subQuery); + } + auto const& resultTable = rbFrag.required("resultTable"); + auto const& jsSubIds = rbFrag.required("subchunkIds"); + for (auto const& scId : jsSubIds) { + fragSubchunkIds.push_back(scId); + } + auto const& jsSubTables = rbFrag.required("subchunkTables"); - UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? + http::RequestBodyJSON rbScDbTable(scDbTable); + string scDb = rbScDbTable.required("scDb"); + string scTable = rbScDbTable.required("scTable"); + TaskDbTbl scDbTbl(scDb, scTable); + fragSubTables.push_back(scDbTbl); + } - /// Make one task for each fragment. - int fragmentCount = taskMsg->fragment_size(); - if (fragmentCount < 1) { - throw util::Bug(ERR_LOC, "Task::createTasks No fragments to execute in TaskMsg"); - } + for (string const& fragSubQ : fragSubQueries) { + size_t templateId = userQueryInfo->addTemplate(fragSubQ); + if (fragSubchunkIds.empty()) { + bool const noSubchunks = false; + int const subchunkId = -1; + auto task = Task::Ptr(new Task( + ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, templateId, + noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats, resultsHttpPort)); - string const chunkIdStr = to_string(taskMsg->chunkid()); - for (int fragNum = 0; fragNum < fragmentCount; ++fragNum) { - proto::TaskMsg_Fragment const& fragment = taskMsg->fragment(fragNum); - for (string queryStr : fragment.query()) { - size_t templateId = userQueryInfo->addTemplate(queryStr); - if (fragment.has_subchunks() && not fragment.subchunks().id().empty()) { - for (auto subchunkId : fragment.subchunks().id()) { - auto task = make_shared(taskMsg, fragNum, userQueryInfo, templateId, - subchunkId, sendChannel, resultsHttpPort); vect.push_back(task); + } else { + for (auto subchunkId : fragSubchunkIds) { + bool const hasSubchunks = true; + auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, + fragmentNumber, templateId, hasSubchunks, subchunkId, + jdQuerySpecDb, scanInfo, scanInteractive, + maxTableSizeMb, fragSubTables, fragSubchunkIds, + sendChannel, queryStats, resultsHttpPort)); + vect.push_back(task); + } } - } else { - int subchunkId = -1; // there are no subchunks. - auto task = make_shared(taskMsg, fragNum, userQueryInfo, templateId, subchunkId, - sendChannel, resultsHttpPort); - vect.push_back(task); } + ++fragmentNumber; } } - for (auto task : vect) { + + for (auto taskPtr : vect) { // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. - task->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(task, chunkResourceMgr, mySqlConfig, - sqlConnMgr, queriesAndChunks)); + taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, + sqlConnMgr, queriesAndChunks)); } - sendChannel->setTaskCount(vect.size()); - return vect; } +*/ -std::vector Task::createTasksForChunk( - std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, proto::ScanInfo const& scanInfo, - bool scanInteractive, int maxTableSizeMb, +std::vector Task::createTasksFromUberJobMsg( + std::shared_ptr const& ujMsg, std::shared_ptr const& ujData, + std::shared_ptr const& sendChannel, std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { QueryId qId = ujData->getQueryId(); UberJobId ujId = ujData->getUberJobId(); + CzarIdType czId = ujData->getCzarId(); + + vector vect; // List of created tasks to be returned. - UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); + UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); string funcN(__func__); funcN += " QID=" + to_string(qId) + " "; + if (ujMsg->getQueryId() != qId) { + throw util::Bug(ERR_LOC, "Task::createTasksFromUberJobMsg qId(" + to_string(qId) + + ") did not match ujMsg->qId(" + to_string(ujMsg->getQueryId()) + + ")"); + } + if (ujMsg->getUberJobId() != ujId) { + throw util::Bug(ERR_LOC, "Task::createTasksFromUberJobMsg ujId(" + to_string(ujId) + + ") did not match ujMsg->qId(" + to_string(ujMsg->getUberJobId()) + + ")"); + } + + std::string workerId = ujMsg->getWorkerId(); + auto jobSubQueryTempMap = ujMsg->getJobSubQueryTempMap(); + auto jobDbTablesMap = ujMsg->getJobDbTablesMap(); + auto jobMsgVect = ujMsg->getJobMsgVect(); + int maxTableSizeMb = ujMsg->getMaxTableSizeMb(); + auto scanInfo = ujMsg->getScanInfo(); + + for (auto const& jobMsg : *jobMsgVect) { + JobId jobId = jobMsg->getJobId(); + int attemptCount = jobMsg->getAttemptCount(); + std::string chunkQuerySpecDb = jobMsg->getChunkQuerySpecDb(); + bool scanInteractive = jobMsg->getScanInteractive(); + int chunkId = jobMsg->getChunkId(); + + std::vector chunkScanTableIndexes = jobMsg->getChunkScanTableIndexes(); + auto jobFragments = jobMsg->getJobFragments(); + int fragmentNumber = 0; + for (auto const& fMsg : *jobFragments) { + // These need to be constructed for the fragment + vector fragSubQueries; + vector fragSubTables; + vector fragSubchunkIds; + + vector fsqIndexes = fMsg->getJobSubQueryTempIndexes(); + for (int fsqIndex : fsqIndexes) { + string fsqStr = jobSubQueryTempMap->getSubQueryTemp(fsqIndex); + fragSubQueries.push_back(fsqStr); + } + + vector dbTblIndexes = fMsg->getJobDbTablesIndexes(); + for (int dbTblIndex : dbTblIndexes) { + auto [scDb, scTable] = jobDbTablesMap->getDbTable(dbTblIndex); + TaskDbTbl scDbTbl(scDb, scTable); + fragSubTables.push_back(scDbTbl); + } + + fragSubchunkIds = fMsg->getSubchunkIds(); + + for (string const& fragSubQ : fragSubQueries) { + size_t templateId = userQueryInfo->addTemplate(fragSubQ); + if (fragSubchunkIds.empty()) { + bool const noSubchunks = false; + int const subchunkId = -1; + auto task = Task::Ptr(new Task( + ujData, jobId, attemptCount, chunkId, fragmentNumber, templateId, noSubchunks, + subchunkId, chunkQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats, resultsHttpPort)); + + vect.push_back(task); + } else { + for (auto subchunkId : fragSubchunkIds) { + bool const hasSubchunks = true; + auto task = Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, + templateId, hasSubchunks, subchunkId, chunkQuerySpecDb, + scanInfo, scanInteractive, maxTableSizeMb, + fragSubTables, fragSubchunkIds, sendChannel, + queryStats, resultsHttpPort)); + vect.push_back(task); + } + } + } + ++fragmentNumber; + } + } + + for (auto taskPtr : vect) { + // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. + taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, + sqlConnMgr, queriesAndChunks)); + } + return vect; +} + +std::vector Task::createTasksForUnitTest( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr) { + QueryId qId = ujData->getQueryId(); + UberJobId ujId = ujData->getUberJobId(); + CzarIdType czId = ujData->getCzarId(); + string funcN(__func__); + funcN += " QID=" + to_string(qId) + " "; + vector vect; for (auto const& job : jsJobs) { json const& jsJobDesc = job["jobdesc"]; - http::RequestBody rbJobDesc(jsJobDesc); + http::RequestBodyJSON rbJobDesc(jsJobDesc); // See qproc::TaskMsgFactory::makeMsgJson for message construction. auto const jdCzarId = rbJobDesc.required("czarId"); auto const jdQueryId = rbJobDesc.required("queryId"); @@ -393,11 +451,11 @@ std::vector Task::createTasksForChunk( vector fragSubchunkIds; vector fragSubTables; LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); - http::RequestBody rbFrag(frag); + http::RequestBodyJSON rbFrag(frag); auto const& jsQueries = rbFrag.required("queries"); // TODO:UJ move to uberjob???, these should be the same for all jobs for (auto const& subQ : jsQueries) { - http::RequestBody rbSubQ(subQ); + http::RequestBodyJSON rbSubQ(subQ); auto const subQuery = rbSubQ.required("subQuery"); LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); fragSubQueries.push_back(subQuery); @@ -410,7 +468,7 @@ std::vector Task::createTasksForChunk( auto const& jsSubTables = rbFrag.required("subchunkTables"); for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? - http::RequestBody rbScDbTable(scDbTable); + http::RequestBodyJSON rbScDbTable(scDbTable); string scDb = rbScDbTable.required("scDb"); string scTable = rbScDbTable.required("scTable"); TaskDbTbl scDbTbl(scDb, scTable); @@ -418,23 +476,23 @@ std::vector Task::createTasksForChunk( } for (string const& fragSubQ : fragSubQueries) { - size_t templateId = userQueryInfo->addTemplate(fragSubQ); if (fragSubchunkIds.empty()) { bool const noSubchunks = false; int const subchunkId = -1; - auto task = Task::Ptr(new Task( - ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, userQueryInfo, - templateId, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, - maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); + auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, + 0, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, + scanInteractive, maxTableSizeMb, fragSubTables, + fragSubchunkIds, sendChannel, nullptr, 0)); + vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; - auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, - fragmentNumber, userQueryInfo, templateId, - hasSubchunks, subchunkId, jdQuerySpecDb, scanInfo, - scanInteractive, maxTableSizeMb, fragSubTables, - fragSubchunkIds, sendChannel, resultsHttpPort)); + auto task = Task::Ptr(new Task( + ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, 0, hasSubchunks, + subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, + fragSubTables, fragSubchunkIds, sendChannel, nullptr, 0)); + vect.push_back(task); } } @@ -443,11 +501,6 @@ std::vector Task::createTasksForChunk( } } - for (auto taskPtr : vect) { - // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. - taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, - sqlConnMgr, queriesAndChunks)); - } return vect; } @@ -484,18 +537,24 @@ void Task::action(util::CmdData* data) { // 'task' contains statistics that are still useful. However, the resources used // by sendChannel need to be freed quickly. LOGS(_log, LOG_LVL_DEBUG, __func__ << " calling resetSendChannel() for " << tIdStr); - resetSendChannel(); // Frees its xrdsvc::SsiRequest object. + resetSendChannel(); // Frees the SendChannel instance } string Task::getQueryString() const { - string qs = _userQueryInfo->getTemplate(_templateId); + auto qStats = _queryStats.lock(); + if (qStats == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _queryStats could not be locked"); + return string(""); + } + + auto uQInfo = qStats->getUserQueryInfo(); + string qs = uQInfo->getTemplate(_templateId); boost::algorithm::replace_all(qs, CHUNK_TAG, to_string(_chunkId)); boost::algorithm::replace_all(qs, SUBCHUNK_TAG, to_string(_subchunkId)); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " qs=" << qs); return qs; } -void Task::setQueryStatistics(wpublish::QueryStatistics::Ptr const& qStats) { _queryStats = qStats; } - wpublish::QueryStatistics::Ptr Task::getQueryStats() const { auto qStats = _queryStats.lock(); if (qStats == nullptr) { @@ -528,14 +587,11 @@ void Task::cancel() { } bool Task::checkCancelled() { - // A czar doesn't directly tell the worker the query is dead. - // A czar has XrdSsi kill the SsiRequest, which kills the - // sendChannel used by this task. sendChannel can be killed - // in other ways, however, without the sendChannel, this task - // has no way to return anything to the originating czar and - // may as well give up now. - if (_sendChannel == nullptr || _sendChannel->isDead()) { - // The sendChannel is dead, probably squashed by the czar. + // The czar does tell the worker a query id is cancelled. + // Returning true here indicates there's no point in doing + // any more processing for this Task. + if (_cancelled) return true; + if (_sendChannel == nullptr || _sendChannel->isDead() || _sendChannel->isRowLimitComplete()) { cancel(); } return _cancelled; @@ -677,23 +733,4 @@ ostream& operator<<(ostream& os, Task const& t) { return os; } -ostream& operator<<(ostream& os, IdSet const& idSet) { - // Limiting output as number of entries can be very large. - int maxDisp = idSet.maxDisp; // only affects the amount of data printed. - lock_guard lock(idSet.mx); - os << "showing " << maxDisp << " of count=" << idSet._ids.size() << " "; - bool first = true; - int i = 0; - for (auto id : idSet._ids) { - if (!first) { - os << ", "; - } else { - first = false; - } - os << id; - if (++i >= maxDisp) break; - } - return os; -} - } // namespace lsst::qserv::wbase diff --git a/src/wbase/Task.h b/src/wbase/Task.h index c8b02f65b2..1f15e7dc64 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -41,7 +41,7 @@ // Qserv headers #include "global/DbTable.h" #include "global/intTypes.h" -#include "proto/ScanTableInfo.h" +#include "protojson/ScanTableInfo.h" #include "wbase/TaskState.h" #include "util/Histogram.h" #include "util/ThreadPool.h" @@ -50,10 +50,11 @@ namespace lsst::qserv::mysql { class MySqlConfig; } -namespace lsst::qserv::proto { -class TaskMsg; -class TaskMsg_Fragment; -} // namespace lsst::qserv::proto + +namespace lsst::qserv::protojson { +class UberJobMsg; +} + namespace lsst::qserv::wbase { class FileChannelShared; } @@ -113,28 +114,6 @@ class TaskScheduler { util::HistogramRolling::Ptr histTimeOfTransmittingTasks; ///< Store information about transmitting tasks. }; -/// Used to find tasks that are in process for debugging with Task::_idStr. -/// This is largely meant to track down incomplete tasks in a possible intermittent -/// failure and should probably be removed when it is no longer needed. -/// It depends on code in BlendScheduler to work. If the decision is made to keep it -/// forever, dependency on BlendScheduler needs to be re-worked. -struct IdSet { // TODO:UJ delete if possible - void add(std::string const& id) { - std::lock_guard lock(mx); - _ids.insert(id); - } - void remove(std::string const& id) { - std::lock_guard lock(mx); - _ids.erase(id); - } - std::atomic maxDisp{5}; //< maximum number of entries to show with operator<< - friend std::ostream& operator<<(std::ostream& os, IdSet const& idSet); - -private: - std::set _ids; - mutable std::mutex mx; -}; - /// class Task defines a query task to be done, containing a TaskMsg /// (over-the-wire) additional concrete info related to physical /// execution conditions. @@ -143,7 +122,6 @@ class Task : public util::CommandForThreadPool { public: static std::string const defaultUser; using Ptr = std::shared_ptr; - using TaskMsgPtr = std::shared_ptr; /// Class to store constant sets and vectors. class DbTblsAndSubchunks { @@ -170,9 +148,8 @@ class Task : public util::CommandForThreadPool { bool operator()(Ptr const& x, Ptr const& y); }; - Task(TaskMsgPtr const& t, int fragmentNumber, std::shared_ptr const& userQueryInfo, - size_t templateId, int subchunkId, std::shared_ptr const& sc, - uint16_t resultsHttpPort = 8080); + std::string cName(const char* func) const { return std::string("Task::") + func; } + // TODO:UJ too many parameters. // - fragmentNumber seems pointless // - hasSubchunks seems redundant. @@ -180,36 +157,49 @@ class Task : public util::CommandForThreadPool { // Candidates: scanInfo, maxTableSizeMb, FileChannelShared, resultsHttpPort. // Unfortunately, this will be much easier if it is done after xrootd method is removed. Task(std::shared_ptr const& ujData, int jobId, int attemptCount, int chunkId, - int fragmentNumber, std::shared_ptr const& userQueryInfo, size_t templateId, - bool hasSubchunks, int subchunkId, std::string const& db, proto::ScanInfo const& scanInfo, - bool scanInteractive, int maxTableSizeMb, std::vector const& fragSubTables, - std::vector const& fragSubchunkIds, std::shared_ptr const& sc, - uint16_t resultsHttpPort = 8080); + int fragmentNumber, size_t templateId, bool hasSubchunks, int subchunkId, std::string const& db, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::vector const& fragSubTables, std::vector const& fragSubchunkIds, + std::shared_ptr const& sc, + std::shared_ptr const& queryStats_, uint16_t resultsHttpPort = 8080); Task& operator=(const Task&) = delete; Task(const Task&) = delete; virtual ~Task(); - /// Read 'taskMsg' to generate a vector of one or more task objects all using the same 'sendChannel' - static std::vector createTasks(std::shared_ptr const& taskMsg, - std::shared_ptr const& sendChannel, - std::shared_ptr const& chunkResourceMgr, - mysql::MySqlConfig const& mySqlConfig, - std::shared_ptr const& sqlConnMgr, - std::shared_ptr const& queriesAndChunks, - uint16_t resultsHttpPort = 8080); - +/* &&& /// Read json to generate a vector of one or more task for a chunk. - static std::vector createTasksForChunk( + static std::vector createTasksForChunk( /// &&& delete std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, proto::ScanInfo const& scanInfo, - bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& sendChannel, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr, + mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, + std::shared_ptr const& queriesAndChunks, + uint16_t resultsHttpPort = 8080); +*/ + + /// &&& + static std::vector createTasksFromUberJobMsg( + std::shared_ptr const& uberJobMsg, + std::shared_ptr const& ujData, + std::shared_ptr const& sendChannel, std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort = 8080); - void setQueryStatistics(std::shared_ptr const& qC); + //&&& + static std::vector createTasksForUnitTest( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr + //&&&mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& + // sqlConnMgr, + //&&&std::shared_ptr const& queriesAndChunks, + //&&&uint16_t resultsHttpPort = 8080); + ); std::shared_ptr getSendChannel() const { return _sendChannel; } void resetSendChannel() { _sendChannel.reset(); } ///< reset the shared pointer for FileChannelShared @@ -222,19 +212,18 @@ class Task : public util::CommandForThreadPool { void action(util::CmdData* data) override; /// Cancel the query in progress and set _cancelled. - /// Query cancellation on the worker is fairly complicated. This - /// function usually called by `SsiRequest::Finished` when xrootd - /// indicates the job is cancelled. This may come from: - /// - xrootd - in the case of communications issues + /// Query cancellation on the worker is fairly complicated. + /// This may come from: /// - czar - user query was cancelled, an error, or limit reached. /// This function may also be called by `Task::checkCancelled()` - `_sendChannel` - /// has been killed, usually a result of failed communication with xrootd. + /// has been killed, usually a result of failed czar communication. /// If a `QueryRunner` object for this task exists, it must /// be cancelled to free up threads and other resources. /// Otherwise `_cancelled` is set so that an attempt /// to run this `Task` will result in a rapid exit. /// This functional also attempts to inform the scheduler for this - /// `Task` that is has been cancelled (scheduler currently does nothing in this case). + /// `Task` that is has been cancelled. The scheduler currently does + /// nothing in this case. void cancel(); /// Check if this task should be cancelled and call cancel() as needed. @@ -264,12 +253,10 @@ class Task : public util::CommandForThreadPool { int getAttemptCount() const { return _attemptCount; } bool getScanInteractive() { return _scanInteractive; } int64_t getMaxTableSize() const { return _maxTableSize; } - proto::ScanInfo& getScanInfo() { return _scanInfo; } + protojson::ScanInfo::Ptr getScanInfo() { return _scanInfo; } void setOnInteractive(bool val) { _onInteractive = val; } bool getOnInteractive() { return _onInteractive; } - static IdSet allIds; // set of all task jobId numbers that are not complete. - /// @return true if qId and jId match this task's query and job ids. bool idsMatch(QueryId qId, int jId, uint64_t tseq) const { return (_qId == qId && _jId == jId && tseq == _tSeq); @@ -339,8 +326,12 @@ class Task : public util::CommandForThreadPool { setFunc(func); } + /// Returns the LIMIT of rows for the query enforceable at the worker, where values <= 0 indicate + /// that there is no limit to the number of rows sent back by the worker. + /// @see UberJobData::getRowLimit() + int getRowLimit() { return _rowLimit; } + private: - std::shared_ptr _userQueryInfo; ///< Details common to Tasks in this UserQuery. std::shared_ptr _sendChannel; ///< Send channel. uint64_t const _tSeq = 0; ///< identifier for the specific task @@ -372,10 +363,14 @@ class Task : public util::CommandForThreadPool { std::atomic _cancelled{false}; TaskQueryRunner::Ptr _taskQueryRunner; std::weak_ptr _taskScheduler; - proto::ScanInfo _scanInfo; + protojson::ScanInfo::Ptr _scanInfo; bool _scanInteractive; ///< True if the czar thinks this query should be interactive. bool _onInteractive{ false}; ///< True if the scheduler put this task on the interactive (group) scheduler. + + /// Stores information on the query's resource usage. + std::weak_ptr const _queryStats; + int64_t _maxTableSize = 0; mutable std::mutex _stateMtx; ///< Mutex to protect state related members _state, _???Time. @@ -389,9 +384,6 @@ class Task : public util::CommandForThreadPool { std::chrono::system_clock::time_point _finishTime; ///< data transmission to Czar fiished size_t _totalSize = 0; ///< Total size of the result so far. - /// Stores information on the query's resource usage. - std::weak_ptr _queryStats; - std::atomic _mysqlThreadId{0}; ///< 0 if not connected to MySQL std::atomic _booted{false}; ///< Set to true if this task takes too long and is booted. @@ -399,9 +391,10 @@ class Task : public util::CommandForThreadPool { /// Time stamp for when `_booted` is set to true, otherwise meaningless. TIMEPOINT _bootedTime; - bool _unitTest = false; ///< + /// When > 0, indicates maximum number of rows needed for a result. + int const _rowLimit; - static std::string const _fqdn; ///< Fully qualified domain name of the host. Acquired once at startup. + bool _unitTest = false; ///< }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 598727a372..a70793f2a4 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -22,6 +22,7 @@ // Class header #include "wbase/UberJobData.h" +#include "../wcontrol/WCzarInfoMap.h" // System headers // Third party headers @@ -34,7 +35,7 @@ #include "http/Exceptions.h" #include "http/MetaModule.h" #include "http/Method.h" -#include "http/RequestBody.h" +#include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" #include "util/Bug.h" #include "util/MultiError.h" @@ -54,14 +55,16 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.UberJobData"); namespace lsst::qserv::wbase { UberJobData::UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, - std::string czarHost, int czarPort, uint64_t queryId, std::string const& workerId, - std::shared_ptr const& foreman, std::string const& authKey) + std::string czarHost, int czarPort, uint64_t queryId, int rowLimit, + std::string const& workerId, std::shared_ptr const& foreman, + std::string const& authKey) : _uberJobId(uberJobId), _czarName(czarName), _czarId(czarId), _czarHost(czarHost), _czarPort(czarPort), _queryId(queryId), + _rowLimit(rowLimit), _workerId(workerId), _authKey(authKey), _foreman(foreman), @@ -76,13 +79,21 @@ void UberJobData::setFileChannelShared(std::shared_ptr const& void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, uint64_t headerCount) { - string const funcN = cName(__func__); LOGS(_log, LOG_LVL_TRACE, - funcN << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize - << " headerCount=" << headerCount); + cName(__func__) << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize + << " headerCount=" << headerCount); + + string workerIdStr; + if (_foreman != nullptr) { + workerIdStr = _foreman->chunkInventory()->id(); + } else { + workerIdStr = "dummyWorkerIdStr"; + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " _foreman was null, which should only happen in unit tests"); + } json request = {{"version", http::MetaModule::version}, - {"workerid", _foreman->chunkInventory()->id()}, + {"workerid", workerIdStr}, {"auth_key", _authKey}, {"czar", _czarName}, {"czarid", _czarId}, @@ -97,36 +108,13 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount vector const headers = {"Content-Type: application/json"}; string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-ready"; string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - http::Client client(method, url, request.dump(), headers); - - int maxTries = 2; // TODO:UJ set from config - bool transmitSuccess = false; - for (int j = 0; (!transmitSuccess && j < maxTries); ++j) { - try { - json const response = client.readAsJson(); - if (0 != response.at("success").get()) { - transmitSuccess = true; - } else { - LOGS(_log, LOG_LVL_WARN, funcN << "Transmit success == 0"); - j = maxTries; /// There's no point in resending as the czar got the message and didn't like - /// it. - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " failed, ex: " + ex.what()); - } - } - - if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, - funcN << "TODO:UJ NEED CODE Let czar find out through polling worker status??? Just throw the " - "result away???"); - } + string const requestStr = request.dump(); + _queueUJResponse(method, headers, url, requestContext, requestStr); } bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled) { - string const funcN = cName(__func__); - LOGS(_log, LOG_LVL_INFO, funcN); + LOGS(_log, LOG_LVL_INFO, cName(__func__)); string errorMsg; int errorCode = 0; if (!multiErr.empty()) { @@ -137,8 +125,8 @@ bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptrgetChunkId()) + ": " + errorMsg; + errorMsg = cName(__func__) + " error(s) in result for chunk #" + to_string(task->getChunkId()) + + ": " + errorMsg; LOGS(_log, LOG_LVL_ERROR, errorMsg); } @@ -156,25 +144,135 @@ bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const headers = {"Content-Type: application/json"}; string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-error"; string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - http::Client client(method, url, request.dump(), headers); + string const requestStr = request.dump(); + _queueUJResponse(method, headers, url, requestContext, requestStr); + return true; +} + +void UberJobData::_queueUJResponse(http::Method method_, std::vector const& headers_, + std::string const& url_, std::string const& requestContext_, + std::string const& requestStr_) { + util::QdispPool::Ptr wPool; + if (_foreman != nullptr) { + wPool = _foreman->getWPool(); + } + + auto cmdTransmit = UJTransmitCmd::create(_foreman, shared_from_this(), method_, headers_, url_, + requestContext_, requestStr_); + if (wPool == nullptr) { + // No thread pool. Run the command now. This should only happen in unit tests. + cmdTransmit->action(nullptr); + } else { + if (_scanInteractive) { + wPool->queCmd(cmdTransmit, 0); + } else { + wPool->queCmd(cmdTransmit, 1); + } + } +} - int maxTries = 2; // TODO:UJ set from config +void UberJobData::cancelAllTasks() { + LOGS(_log, LOG_LVL_INFO, cName(__func__)); + if (_cancelled.exchange(true) == false) { + lock_guard lg(_ujTasksMtx); + for (auto const& task : _ujTasks) { + task->cancel(); + } + } +} + +string UJTransmitCmd::cName(const char* funcN) const { + stringstream os; + os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " qId=" << _queryId << " ujId=" << _uberJobId; + return os.str(); +} + +void UJTransmitCmd::action(util::CmdData* data) { + // Make certain _selfPtr is reset before leaving this function. + // If a retry is needed, duplicate() is called. + class ResetSelf { + public: + ResetSelf(UJTransmitCmd* ujtCmd) : _ujtCmd(ujtCmd) {} + ~ResetSelf() { _ujtCmd->_selfPtr.reset(); } + UJTransmitCmd* const _ujtCmd; + }; + ResetSelf resetSelf(this); + + _attemptCount++; + auto ujPtr = _ujData.lock(); + if (ujPtr == nullptr || ujPtr->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " UberJob was cancelled " << _attemptCount); + return; + } + http::Client client(_method, _url, _requestStr, _headers); bool transmitSuccess = false; - for (int j = 0; !transmitSuccess && j < maxTries; ++j) { - try { - json const response = client.readAsJson(); - if (0 != response.at("success").get()) { - transmitSuccess = true; - } else { - LOGS(_log, LOG_LVL_WARN, funcN << " transmit success == 0"); - j = maxTries; /// There's no point in resending as the czar got the message and didn't like - /// it. + try { + json const response = client.readAsJson(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " Transmit success == 0"); + // There's no point in re-sending as the czar got the message and didn't like + // it. + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " " + _requestContext + " failed, ex: " + ex.what()); + } + + if (!transmitSuccess) { + auto sPtr = _selfPtr; + if (_foreman != nullptr && sPtr != nullptr) { + // Do not reset _selfPtr as re-queuing may be needed several times. + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " no response for transmit, putting on failed transmit queue."); + auto wCzInfo = _foreman->getWCzarInfoMap()->getWCzarInfo(_czarId); + // This will check if the czar is believed to be alive and try the queue the query to be tried + // again at a lower priority. It it thinks the czar is dead, it will throw it away. + // TODO:UJ &&& I have my doubts about this as a reconnected czar may go down in flames + // &&& as it is hit with thousands of these. + // &&& Alternate plan, set a flag in the status message response (WorkerQueryStatusData) + // &&& indicates some messages failed. When the czar sees the flag, it'll request a + // &&& message from the worker that contains all of the failed transmit data and handle + // &&& that. All of these failed transmits should fit in a single message. + if (wCzInfo->checkAlive(CLOCK::now())) { + auto wPool = _foreman->getWPool(); + if (wPool != nullptr) { + Ptr replacement = duplicate(); + if (replacement != nullptr) { + wPool->queCmd(replacement, 2); + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " replacement was null"); + } + } else { + // No thread pool, should only be possible in unit tests. + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no wPool"); + return; + } } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " failed, ex: " + ex.what()); + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _selfPtr was null, assuming job killed."); } } - return transmitSuccess; +} + +void UJTransmitCmd::kill() { + string const funcN("UJTransmitCmd::kill"); + LOGS(_log, LOG_LVL_WARN, funcN); + auto sPtr = _selfPtr; + _selfPtr.reset(); + if (sPtr == nullptr) { + return; + } +} + +UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { + auto ujD = _ujData.lock(); + if (ujD == nullptr) { + return nullptr; + } + Ptr newPtr = create(_foreman, ujD, _method, _headers, _url, _requestContext, _requestStr); + newPtr->_attemptCount = _attemptCount; + return newPtr; } } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index f4ab4e3030..d4765fbbe7 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -34,7 +34,9 @@ // Qserv headers #include "global/intTypes.h" +#include "http/Method.h" #include "qmeta/types.h" +#include "util/QdispPool.h" #include "wbase/SendChannel.h" namespace lsst::qserv { @@ -55,7 +57,7 @@ class Task; /// This class tracks all Tasks associates with the UberJob on the worker /// and reports status to the czar. -class UberJobData { +class UberJobData : public std::enable_shared_from_this { public: using Ptr = std::shared_ptr; @@ -63,15 +65,17 @@ class UberJobData { UberJobData(UberJobData const&) = delete; static Ptr create(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, - std::string const& czarHost, int czarPort, uint64_t queryId, + std::string const& czarHost, int czarPort, uint64_t queryId, int rowLimit, std::string const& workerId, std::shared_ptr const& foreman, std::string const& authKey) { - return Ptr(new UberJobData(uberJobId, czarName, czarId, czarHost, czarPort, queryId, workerId, - foreman, authKey)); + return Ptr(new UberJobData(uberJobId, czarName, czarId, czarHost, czarPort, queryId, rowLimit, + workerId, foreman, authKey)); } /// Set file channel for this UberJob void setFileChannelShared(std::shared_ptr const& fileChannelShared); + void setScanInteractive(bool scanInteractive) { _scanInteractive = scanInteractive; } + UberJobId getUberJobId() const { return _uberJobId; } qmeta::CzarId getCzarId() const { return _czarId; } std::string getCzarHost() const { return _czarHost; } @@ -81,6 +85,7 @@ class UberJobData { /// Add the tasks defined in the UberJob to this UberJobData object. void addTasks(std::vector> const& tasks) { + std::lock_guard tLg(_ujTasksMtx); _ujTasks.insert(_ujTasks.end(), tasks.begin(), tasks.end()); } @@ -94,17 +99,34 @@ class UberJobData { std::string getIdStr() const { return _idStr; } std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } + bool getCancelled() const { return _cancelled; } + + /// Cancel all Tasks in this UberJob. + void cancelAllTasks(); + + /// Returns the LIMIT of rows for the query enforceable at the worker, where values <= 0 indicate + /// that there is no limit to the number of rows sent back by the worker. + /// Workers can only safely limit rows for queries that have the LIMIT clause without other related + /// clauses like ORDER BY. + int getRowLimit() { return _rowLimit; } + private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, - int czarPort, uint64_t queryId, std::string const& workerId, + int czarPort, uint64_t queryId, int rowLimit, std::string const& workerId, std::shared_ptr const& foreman, std::string const& authKey); + /// Queue the response to be sent to the originating czar. + void _queueUJResponse(http::Method method_, std::vector const& headers_, + std::string const& url_, std::string const& requestContext_, + std::string const& requestStr_); + UberJobId const _uberJobId; std::string const _czarName; qmeta::CzarId const _czarId; std::string const _czarHost; int const _czarPort; QueryId const _queryId; + int const _rowLimit; ///< If > 0, only read this many rows before return the results. std::string const _workerId; std::string const _authKey; @@ -113,7 +135,80 @@ class UberJobData { std::vector> _ujTasks; std::shared_ptr _fileChannelShared; + std::mutex _ujTasksMtx; ///< Protects _ujTasks. + std::string const _idStr; + + /// True if this an interactive (aka high priority) user query. + std::atomic _scanInteractive; + + std::atomic _cancelled{false}; ///< Set to true if this was cancelled. +}; + +/// This class puts the information about a locally finished UberJob into a command +/// so it can be put on a queue and sent to the originating czar. The information +/// being transmitted is usually the url for the result file or an error message. +class UJTransmitCmd : public util::PriorityCommand { +public: + using Ptr = std::shared_ptr; + + UJTransmitCmd() = delete; + ~UJTransmitCmd() override = default; + + std::string cName(const char* funcN) const; + + static Ptr create(std::shared_ptr const& foreman_, UberJobData::Ptr const& ujData_, + http::Method method_, std::vector const& headers_, std::string const& url_, + std::string const& requestContext_, std::string const& requestStr_) { + auto ptr = Ptr( + new UJTransmitCmd(foreman_, ujData_, method_, headers_, url_, requestContext_, requestStr_)); + ptr->_selfPtr = ptr; + return ptr; + } + + /// Send the UberJob file to the czar, this is the function that will be run when + /// the queue reaches this command. If this message is not received by the czar, + /// it will notify WCzarInfo and possibly send WorkerCzarComIssue. + void action(util::CmdData* data) override; + + /// Reset the self pointer so this object can be killed. + void kill(); + + /// This function makes a duplicate of the required information for transmition to the czar + /// in a new object and then increments the attempt count, so it is not a true copy. + /// Priority commands cannot be resent as there's information in them about which queue + /// to modify, so a fresh object is needed to re-send. The message and target czar remain + /// unchanged except for the atttempt count. + Ptr duplicate(); + +private: + UJTransmitCmd(std::shared_ptr const& foreman_, UberJobData::Ptr const& ujData_, + http::Method method_, std::vector const& headers_, std::string const& url_, + std::string const& requestContext_, std::string const& requestStr_) + : PriorityCommand(), + _foreman(foreman_), + _ujData(ujData_), + _czarId(ujData_->getCzarId()), + _queryId(ujData_->getQueryId()), + _uberJobId(ujData_->getUberJobId()), + _method(method_), + _headers(headers_), + _url(url_), + _requestContext(requestContext_), + _requestStr(requestStr_) {} + + Ptr _selfPtr; ///< So this object can put itself back on the queue and keep itself alive. + std::shared_ptr const _foreman; + std::weak_ptr const _ujData; + CzarIdType const _czarId; + QueryId const _queryId; + UberJobId const _uberJobId; + http::Method const _method; + std::vector const _headers; + std::string const _url; + std::string const _requestContext; + std::string const _requestStr; + int _attemptCount = 0; ///< How many attempts have been made to transmit this. }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/UserQueryInfo.cc b/src/wbase/UserQueryInfo.cc index 79c24f07ed..888180088c 100644 --- a/src/wbase/UserQueryInfo.cc +++ b/src/wbase/UserQueryInfo.cc @@ -37,46 +37,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.UserQueryInfo"); namespace lsst::qserv::wbase { -UserQueryInfo::UserQueryInfo(QueryId qId) : _qId(qId) {} - -UserQueryInfo::Ptr UserQueryInfo::uqMapInsert(QueryId qId) { - Ptr uqi; - lock_guard lg(_uqMapMtx); - auto iter = _uqMap.find(qId); - if (iter != _uqMap.end()) { - uqi = iter->second.lock(); - } - // If uqi is invalid at this point, a new one needs to be made. - if (uqi == nullptr) { - uqi = make_shared(qId); - _uqMap[qId] = uqi; - } - return uqi; -} - -UserQueryInfo::Ptr UserQueryInfo::uqMapGet(QueryId qId) { - lock_guard lg(_uqMapMtx); - auto iter = _uqMap.find(qId); - if (iter != _uqMap.end()) { - return iter->second.lock(); - } - return nullptr; -} - -void UserQueryInfo::uqMapErase(QueryId qId) { - lock_guard lg(_uqMapMtx); - auto iter = _uqMap.find(qId); - if (iter != _uqMap.end()) { - // If the weak pointer has 0 real references - if (iter->second.expired()) { - _uqMap.erase(qId); - } - } -} - -UserQueryInfo::Map UserQueryInfo::_uqMap; - -mutex UserQueryInfo::_uqMapMtx; +UserQueryInfo::UserQueryInfo(QueryId qId, CzarIdType czarId) : _qId(qId), _czarId(czarId) {} size_t UserQueryInfo::addTemplate(std::string const& templateStr) { size_t j = 0; @@ -108,4 +69,50 @@ void UserQueryInfo::addUberJob(std::shared_ptr const& ujData) { _uberJobMap[ujId] = ujData; } +void UserQueryInfo::cancelFromCzar() { + if (_cancelledByCzar.exchange(true)) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " already cancelledByCzar"); + return; + } + lock_guard lockUq(_uberJobMapMtx); + for (auto const& [ujId, weakUjPtr] : _uberJobMap) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " cancelling ujId=" << ujId); + auto ujPtr = weakUjPtr.lock(); + if (ujPtr != nullptr) { + ujPtr->cancelAllTasks(); + } + } +} + +void UserQueryInfo::cancelUberJob(UberJobId ujId) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " cancelling ujId=" << ujId); + lock_guard lockUq(_uberJobMapMtx); + _deadUberJobSet.insert(ujId); + auto iter = _uberJobMap.find(ujId); + if (iter != _uberJobMap.end()) { + auto weakUjPtr = iter->second; + auto ujPtr = weakUjPtr.lock(); + if (ujPtr != nullptr) { + ujPtr->cancelAllTasks(); + } + } +} + +void UserQueryInfo::cancelAllUberJobs() { + lock_guard lockUq(_uberJobMapMtx); + for (auto const& [ujKey, weakUjPtr] : _uberJobMap) { + _deadUberJobSet.insert(ujKey); + auto ujPtr = weakUjPtr.lock(); + if (ujPtr != nullptr) { + ujPtr->cancelAllTasks(); + } + } +} + +bool UserQueryInfo::isUberJobDead(UberJobId ujId) const { + lock_guard lockUq(_uberJobMapMtx); + auto iter = _deadUberJobSet.find(ujId); + return iter != _deadUberJobSet.end(); +} + } // namespace lsst::qserv::wbase diff --git a/src/wbase/UserQueryInfo.h b/src/wbase/UserQueryInfo.h index 4b7a799f03..eb15de7089 100644 --- a/src/wbase/UserQueryInfo.h +++ b/src/wbase/UserQueryInfo.h @@ -24,6 +24,7 @@ #define LSST_QSERV_WBASE_USERQUERYINFO_H // System headers +#include #include #include #include @@ -44,20 +45,20 @@ class UserQueryInfo { using Ptr = std::shared_ptr; using Map = std::map>; - static Ptr uqMapInsert(QueryId qId); - static Ptr uqMapGet(QueryId qId); - /// Erase the entry for `qId` in the map, as long as there are only - /// weak references to the UserQueryInfoObject. - /// Clear appropriate local and member references before calling this. - static void uqMapErase(QueryId qId); - - UserQueryInfo(QueryId qId); UserQueryInfo() = delete; UserQueryInfo(UserQueryInfo const&) = delete; UserQueryInfo& operator=(UserQueryInfo const&) = delete; + static Ptr create(QueryId qId, CzarIdType czarId) { + return std::shared_ptr(new UserQueryInfo(qId, czarId)); + } + ~UserQueryInfo() = default; + std::string cName(const char* func) { + return std::string("UserQueryInfo::") + func + " qId=" + std::to_string(_qId); + } + /// Add a query template to the map of templates for this user query. size_t addTemplate(std::string const& templateStr); @@ -68,21 +69,45 @@ class UserQueryInfo { /// Add an UberJobData object to the UserQueryInfo. void addUberJob(std::shared_ptr const& ujData); + /// Return true if this user query was cancelled by its czar. + bool getCancelledByCzar() const { return _cancelledByCzar; } + + /// The czar has cancelled this user query, all tasks need to + /// be killed but there's no need to track UberJob id's anymore. + void cancelFromCzar(); + + /// Cancel all associated tasks and track the killed UberJob id's + /// The user query itself may still be alive, so the czar may need + /// information about which UberJobs are dead. + void cancelAllUberJobs(); + + /// Cancel a specific UberJob in this user query. + void cancelUberJob(UberJobId ujId); + + bool isUberJobDead(UberJobId ujId) const; + + QueryId getQueryId() const { return _qId; } + + CzarIdType getCzarId() const { return _czarId; } + private: - static Map _uqMap; - static std::mutex _uqMapMtx; ///< protects _uqMap + UserQueryInfo(QueryId qId, CzarIdType czId); QueryId const _qId; ///< The User Query Id number. + CzarIdType const _czarId; /// List of template strings. This is expected to be short, 1 or 2 entries. /// This must be a vector. New entries are always added to the end so as not /// to alter existing indexes into the vector. std::vector _templates; - std::mutex _uqMtx; ///< protects _templates; + std::mutex _uqMtx; ///< protects _templates /// Map of all UberJobData objects on this worker for this User Query. - std::map> _uberJobMap; - std::mutex _uberJobMapMtx; ///< protects _uberJobMap; + std::map> _uberJobMap; + std::set _deadUberJobSet; ///< Set of cancelled UberJob Ids. + mutable std::mutex _uberJobMapMtx; ///< protects _uberJobMap, _deadUberJobSet + + std::atomic _cancelledByCzar{false}; }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/WorkerCommand.cc b/src/wbase/WorkerCommand.cc deleted file mode 100644 index cf79089a92..0000000000 --- a/src/wbase/WorkerCommand.cc +++ /dev/null @@ -1,49 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2012-2018 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "wbase/WorkerCommand.h" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "wbase/SendChannel.h" - -namespace { - -LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.WorkerCommand"); - -} // namespace - -namespace lsst::qserv::wbase { - -WorkerCommand::WorkerCommand(SendChannel::Ptr const& sendChannel) - : util::Command([this](util::CmdData* data) { this->run(); }), _sendChannel(sendChannel) {} - -void WorkerCommand::sendSerializedResponse() { - std::string str(_frameBuf.data(), _frameBuf.size()); - _sendChannel->sendStream(xrdsvc::StreamBuffer::createWithMove(str), true); -} - -} // namespace lsst::qserv::wbase diff --git a/src/wbase/WorkerCommand.h b/src/wbase/WorkerCommand.h deleted file mode 100644 index c0934f4797..0000000000 --- a/src/wbase/WorkerCommand.h +++ /dev/null @@ -1,96 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2011-2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// WorkerCommand.h -#ifndef LSST_QSERV_WBASE_WORKER_COMMAND_H -#define LSST_QSERV_WBASE_WORKER_COMMAND_H - -// System headers -#include -#include -#include -#include - -// Qserv headers -#include "proto/FrameBuffer.h" -#include "proto/worker.pb.h" -#include "util/Command.h" - -// Forward declarations -namespace lsst::qserv::wbase { -class SendChannel; -} // namespace lsst::qserv::wbase - -namespace lsst::qserv::wbase { - -/** - * Class WorkerCommand is the base class for a family of various worker - * management commmands. - */ -class WorkerCommand : public util::Command { -public: - using Ptr = std::shared_ptr; - - WorkerCommand& operator=(const WorkerCommand&) = delete; - WorkerCommand(const WorkerCommand&) = delete; - WorkerCommand() = delete; - virtual ~WorkerCommand() = default; - - /// @param sendChannel - communication channel for reporting results - explicit WorkerCommand(std::shared_ptr const& sendChannel); - -protected: - /// The actual behavior is provided by subclasses. - virtual void run() = 0; - - /** - * Fill in the status code and the message into the response message - * of the desired type and sent it back to a caller. - * @param error Mandatory error to be reported. - * @param code The optional error code if the one differes from the default one. - * @param extendedModsFunc The optional function to be provided if any additional modifications - * are required to be made to the response object. - */ - template - void reportError(std::string const& error, - proto::WorkerCommandStatus::Code code = proto::WorkerCommandStatus::ERROR, - std::function const& extendedModsFunc = nullptr) { - RESPONSE resp; - resp.mutable_status()->set_code(code); - resp.mutable_status()->set_error(error); - if (extendedModsFunc != nullptr) extendedModsFunc(resp); - _frameBuf.serialize(resp); - sendSerializedResponse(); - } - - /** - * Send the serialized payload stored within the frame buffer to a caller. - */ - void sendSerializedResponse(); - - std::shared_ptr _sendChannel; ///< For result reporting - proto::FrameBuffer _frameBuf; ///< Buffer for serializing a response -}; - -} // namespace lsst::qserv::wbase - -#endif // LSST_QSERV_WBASE_WORKER_COMMAND_H diff --git a/src/wconfig/WorkerConfig.h b/src/wconfig/WorkerConfig.h index c7a563567a..bb4ddf2b0d 100644 --- a/src/wconfig/WorkerConfig.h +++ b/src/wconfig/WorkerConfig.h @@ -162,6 +162,29 @@ class WorkerConfig { /// @return the port number of the worker XROOTD service for serving result files uint16_t resultsXrootdPort() const { return _resultsXrootdPort->getVal(); } + /// The size + int getQPoolSize() const { return _qPoolSize->getVal(); } + + /// The highest priority number, such as 2, which results + /// in queues for priorities 0, 1, 2, and 100; where 0 is the + /// highest priority. + /// @see util::QdispPool + int getQPoolMaxPriority() const { return _qPoolMaxPriority->getVal(); } + + /// The maximum number of running threads at each priority, + /// "30:20:20:10" with _qPoolMaxPriority=2 allows 30 threads + /// at priority 0, 20 threads at priorities 1+2, and 10 threads + /// at priority 100. + /// @see util::QdispPool + std::string getQPoolRunSizes() const { return _qPoolRunSizes->getVal(); } + + /// The minimum number of running threads per priority, + /// "3:3:3:3" with _qPoolMaxPriority=2 means that a thread at priority + /// 0 would not start if it meant that there would not be enough threads + /// left to have running for each of priorities 1, 2, and 100. + /// @see util::QdispPool + std::string getQPoolMinRunningSizes() const { return _qPoolMinRunningSizes->getVal(); } + /// @return the number of the BOOST ASIO threads for servicing HTGTP requests size_t resultsNumHttpThreads() const { return _resultsNumHttpThreads->getVal(); } @@ -336,6 +359,14 @@ class WorkerConfig { CVTStrPtr _httpUser = util::ConfigValTStr::create(_configValMap, "http", "user", notReq, ""); CVTStrPtr _httpPassword = util::ConfigValTStr::create(_configValMap, "http", "password", notReq, "", hidden); + + CVTIntPtr _qPoolSize = util::ConfigValTInt::create(_configValMap, "qpool", "Size", notReq, 50); + CVTIntPtr _qPoolMaxPriority = + util::ConfigValTInt::create(_configValMap, "qpool", "MaxPriority", notReq, 2); + CVTStrPtr _qPoolRunSizes = + util::ConfigValTStr::create(_configValMap, "qpool", "RunSizes", notReq, "30:20:20:10"); + CVTStrPtr _qPoolMinRunningSizes = + util::ConfigValTStr::create(_configValMap, "qpool", "MinRunningSizes", notReq, "3:3:3:3"); }; } // namespace lsst::qserv::wconfig diff --git a/src/wcontrol/CMakeLists.txt b/src/wcontrol/CMakeLists.txt index 24600c50e3..ec8729388b 100644 --- a/src/wcontrol/CMakeLists.txt +++ b/src/wcontrol/CMakeLists.txt @@ -6,6 +6,7 @@ target_sources(wcontrol PRIVATE ResourceMonitor.cc SqlConnMgr.cc WorkerStats.cc + WCzarInfoMap.cc ) install( diff --git a/src/wcontrol/Foreman.cc b/src/wcontrol/Foreman.cc index 84b9a44ef4..a5b90f96e1 100644 --- a/src/wcontrol/Foreman.cc +++ b/src/wcontrol/Foreman.cc @@ -39,10 +39,13 @@ #include "qhttp/Response.h" #include "qhttp/Server.h" #include "qhttp/Status.h" -#include "wbase/WorkerCommand.h" +#include "util/common.h" +#include "util/QdispPool.h" +#include "util/String.h" #include "wconfig/WorkerConfig.h" #include "wcontrol/ResourceMonitor.h" #include "wcontrol/SqlConnMgr.h" +#include "wcontrol/WCzarInfoMap.h" #include "wcontrol/WorkerStats.h" #include "wdb/ChunkResource.h" #include "wdb/SQLBackend.h" @@ -78,10 +81,32 @@ qhttp::Status removeResultFile(std::string const& fileName) { namespace lsst::qserv::wcontrol { +Foreman::Ptr Foreman::_globalForeman; + +Foreman::Ptr Foreman::create(Scheduler::Ptr const& scheduler, unsigned int poolSize, + unsigned int maxPoolThreads, mysql::MySqlConfig const& mySqlConfig, + wpublish::QueriesAndChunks::Ptr const& queries, + std::shared_ptr const& chunkInventory, + std::shared_ptr const& sqlConnMgr, int qPoolSize, + int maxPriority, std::string const& vectRunSizesStr, + std::string const& vectMinRunningSizesStr) { + // Latch + static std::atomic globalForemanSet{false}; + if (globalForemanSet.exchange(true) == true) { + throw util::Bug(ERR_LOC, "Foreman::create already an existing global Foreman."); + } + + Ptr fm = Ptr(new Foreman(scheduler, poolSize, maxPoolThreads, mySqlConfig, queries, chunkInventory, + sqlConnMgr, qPoolSize, maxPriority, vectRunSizesStr, vectMinRunningSizesStr)); + _globalForeman = fm; + return _globalForeman; +} + Foreman::Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, mysql::MySqlConfig const& mySqlConfig, wpublish::QueriesAndChunks::Ptr const& queries, std::shared_ptr const& chunkInventory, - std::shared_ptr const& sqlConnMgr) + std::shared_ptr const& sqlConnMgr, int qPoolSize, int maxPriority, + std::string const& vectRunSizesStr, std::string const& vectMinRunningSizesStr) : _scheduler(scheduler), _mySqlConfig(mySqlConfig), _queries(queries), @@ -89,7 +114,8 @@ Foreman::Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigne _sqlConnMgr(sqlConnMgr), _resourceMonitor(make_shared()), _io_service(), - _httpServer(qhttp::Server::create(_io_service, 0 /* grab the first available port */)) { + _httpServer(qhttp::Server::create(_io_service, 0 /* grab the first available port */)), + _wCzarInfoMap(WCzarInfoMap::create()) { // Make the chunk resource mgr // Creating backend makes a connection to the database for making temporary tables. // It will delete temporary tables that it can identify as being created by a worker. @@ -109,6 +135,15 @@ Foreman::Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigne _mark = make_shared(ERR_LOC, "Forman Test Msg"); + vector vectRunSizes = util::String::parseToVectInt(vectRunSizesStr, ":", 1); + vector vectMinRunningSizes = util::String::parseToVectInt(vectMinRunningSizesStr, ":", 0); + LOGS(_log, LOG_LVL_INFO, + "INFO wPool config qPoolSize=" << qPoolSize << " maxPriority=" << maxPriority << " vectRunSizes=" + << vectRunSizesStr << " -> " << util::prettyCharList(vectRunSizes) + << " vectMinRunningSizes=" << vectMinRunningSizesStr << " -> " + << util::prettyCharList(vectMinRunningSizes)); + _wPool = make_shared(qPoolSize, maxPriority, vectRunSizes, vectMinRunningSizes); + // Read-only access to the result files via the HTTP protocol's method "GET" auto const workerConfig = wconfig::WorkerConfig::instance(); _httpServer->addStaticContent("/*", workerConfig->resultsDirname()); @@ -148,10 +183,6 @@ void Foreman::processTasks(vector const& tasks) { _scheduler->queCmd(cmds); } -void Foreman::processCommand(shared_ptr const& command) { - _workerCommandQueue->queCmd(command); -} - uint16_t Foreman::httpPort() const { return _httpServer->getPort(); } nlohmann::json Foreman::statusToJson(wbase::TaskSelector const& taskSelector) { diff --git a/src/wcontrol/Foreman.h b/src/wcontrol/Foreman.h index 17fd0f14f6..ed2f78518a 100644 --- a/src/wcontrol/Foreman.h +++ b/src/wcontrol/Foreman.h @@ -39,8 +39,8 @@ #include "mysql/MySqlConfig.h" #include "util/EventThread.h" #include "util/HoldTrack.h" +#include "util/QdispPool.h" #include "wbase/Base.h" -#include "wbase/MsgProcessor.h" #include "wbase/Task.h" // Forward declarations @@ -50,6 +50,7 @@ struct TaskSelector; } // namespace lsst::qserv::wbase namespace lsst::qserv::wcontrol { +class WCzarInfoMap; class ResourceMonitor; class SqlConnMgr; } // namespace lsst::qserv::wcontrol @@ -66,6 +67,7 @@ class QueryRunner; namespace lsst::qserv::wpublish { class ChunkInventory; class QueriesAndChunks; +class QueryStatistics; } // namespace lsst::qserv::wpublish // This header declarations @@ -95,8 +97,12 @@ class Scheduler : public wbase::TaskScheduler, public util::CommandQueue { /// Foreman is used to maintain a thread pool and schedule Tasks for the thread pool. /// It also manages sub-chunk tables with the ChunkResourceMgr. /// The schedulers may limit the number of threads they will use from the thread pool. -class Foreman : public wbase::MsgProcessor { +class Foreman { public: + using Ptr = std::shared_ptr; + + static Ptr getForeman() { return _globalForeman; } + /** * @param scheduler - pointer to the scheduler * @param poolSize - size of the thread pool @@ -105,12 +111,14 @@ class Foreman : public wbase::MsgProcessor { * @param chunkInventory - a collection of the SSI resources published by the worker * @param sqlConnMgr - for limiting the number of MySQL connections used for tasks */ - Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, - mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& queries, - std::shared_ptr const& chunkInventory, - std::shared_ptr const& sqlConnMgr); + static Ptr create(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, + mysql::MySqlConfig const& mySqlConfig, + std::shared_ptr const& queries, + std::shared_ptr const& chunkInventory, + std::shared_ptr const& sqlConnMgr, int qPoolSize, int maxPriority, + std::string const& vectRunSizesStr, std::string const& vectMinRunningSizesStr); - virtual ~Foreman() override; + ~Foreman(); // This class doesn't have the default construction or copy semantics Foreman() = delete; @@ -127,18 +135,30 @@ class Foreman : public wbase::MsgProcessor { uint16_t httpPort() const; /// Process a group of query processing tasks. - /// @see MsgProcessor::processTasks() - void processTasks(std::vector> const& tasks) override; + void processTasks(std::vector> const& tasks); /// Implement the corresponding method of the base class - /// @see MsgProcessor::processCommand() - void processCommand(std::shared_ptr const& command) override; + nlohmann::json statusToJson(wbase::TaskSelector const& taskSelector); - /// Implement the corresponding method of the base class - /// @see MsgProcessor::statusToJson() - virtual nlohmann::json statusToJson(wbase::TaskSelector const& taskSelector) override; + uint64_t getWorkerStartupTime() const { return _workerStartupTime; } + + std::shared_ptr getWPool() const { return _wPool; } + + std::shared_ptr getWCzarInfoMap() const { return _wCzarInfoMap; } + + std::shared_ptr getQueriesAndChunks() const { return _queries; } private: + Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, + mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& queries, + std::shared_ptr const& chunkInventory, + std::shared_ptr const& sqlConnMgr, int qPoolSize, int maxPriority, + std::string const& vectRunSizesStr, std::string const& vectMinRunningSizesStr); + + /// Startup time of worker, sent to czars so they can detect that the worker was + /// was restarted when this value changes. + uint64_t const _workerStartupTime = millisecSinceEpoch(CLOCK::now()); + std::shared_ptr _chunkResourceMgr; util::ThreadPool::Ptr _pool; @@ -165,6 +185,19 @@ class Foreman : public wbase::MsgProcessor { /// The HTTP server for serving/managing result files std::shared_ptr const _httpServer; + + /// Combined priority queue and thread pool for communicating with czars. + /// TODO:UJ - It would be better to have a pool for each czar as it + /// may be possible for a czar to have communications + /// problems in a way that would wedge the pool. This can + /// probably be done fairly easily by having pools + /// attached to wcontrol::WCzarInfoMap. + std::shared_ptr _wPool; + + /// Map of czar information for all czars that have contacted this worker. + std::shared_ptr const _wCzarInfoMap; + + static Ptr _globalForeman; ///< Pointer to the global instance. }; } // namespace lsst::qserv::wcontrol diff --git a/src/wcontrol/WCzarInfoMap.cc b/src/wcontrol/WCzarInfoMap.cc new file mode 100644 index 0000000000..831022c44d --- /dev/null +++ b/src/wcontrol/WCzarInfoMap.cc @@ -0,0 +1,186 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "wcontrol/WCzarInfoMap.h" + +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "http/Client.h" +#include "protojson/WorkerQueryStatusData.h" +#include "util/Bug.h" +#include "util/Histogram.h" +#include "wbase/UberJobData.h" +#include "wconfig/WorkerConfig.h" +#include "wcontrol/Foreman.h" +#include "wpublish/QueriesAndChunks.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; + +using namespace std::chrono_literals; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.wcontrol.WCzarInfoMap"); +} + +namespace lsst::qserv::wcontrol { + +WCzarInfo::WCzarInfo(CzarIdType czarId_) + : czarId(czarId_), + _workerCzarComIssue(protojson::WorkerCzarComIssue::create( + wconfig::WorkerConfig::instance()->replicationInstanceId(), + wconfig::WorkerConfig::instance()->replicationAuthKey())) {} + +void WCzarInfo::czarMsgReceived(TIMEPOINT tm) { + unique_lock uniLock(_wciMtx); + _lastTouch = tm; + if (_alive.exchange(true) == false) { + uniLock.unlock(); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " was dead and is now alive"); + _workerCzarComIssue->setThoughtCzarWasDead(true); + } +} + +void WCzarInfo::sendWorkerCzarComIssueIfNeeded(protojson::WorkerContactInfo::Ptr const& wInfo_, + protojson::CzarContactInfo::Ptr const& czInfo_) { + unique_lock uniLock(_wciMtx); + if (_workerCzarComIssue->needToSend()) { + // Having more than one of this message being sent at one time + // could cause race issues and it would be a problem if it was + // stuck in a queue, so it gets its own thread. + if (_msgThreadRunning.exchange(true) == true) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " message thread already running"); + return; + } + _workerCzarComIssue->setContactInfo(wInfo_, czInfo_); + auto selfPtr = weak_from_this(); + auto thrdFunc = [selfPtr]() { + auto sPtr = selfPtr.lock(); + if (sPtr == nullptr) { + LOGS(_log, LOG_LVL_WARN, "WCzarInfo::sendWorkerCzarComIssueIfNeeded thrdFunc sPtr was null"); + } + sPtr->_sendMessage(); + }; + + thread thrd(thrdFunc); + thrd.detach(); + } +} + +void WCzarInfo::_sendMessage() { + // Make certain _msgThreadRunning is set to false when this function ends. + class ClearMsgThreadRunning { + public: + ClearMsgThreadRunning(WCzarInfo* wcInfo) : _wcInfo(wcInfo) {} + ~ClearMsgThreadRunning() { _wcInfo->_msgThreadRunning = false; } + WCzarInfo* const _wcInfo; + }; + ClearMsgThreadRunning clearMsgThreadRunning(this); + + auto const method = http::Method::POST; + + unique_lock uniLock(_wciMtx); + auto czInfo = _workerCzarComIssue->getCzarInfo(); + // If thoughtCzarWasDead is set now, it needs to be cleared on successful reception from czar. + bool needToClearThoughtCzarWasDead = _workerCzarComIssue->getThoughtCzarWasDead(); + if (czInfo == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " czar info was null"); + return; + } + vector const headers = {"Content-Type: application/json"}; + string const url = + "http://" + czInfo->czHostName + ":" + to_string(czInfo->czPort) + "/workerczarcomissue"; + auto jsReqPtr = _workerCzarComIssue->serializeJson(); + uniLock.unlock(); // Must unlock before communication + + auto requestStr = jsReqPtr->dump(); + http::Client client(method, url, requestStr, headers); + bool transmitSuccess = false; + try { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read start"); + nlohmann::json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read end"); + uniLock.lock(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + if (needToClearThoughtCzarWasDead) { + _workerCzarComIssue->setThoughtCzarWasDead(false); + } + } else { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " Transmit success == 0"); + // There's no point in re-sending as the czar got the message and didn't like + // it. + // TODO:UJ &&& maybe add this czId+ujId to a list of failed uberjobs that can be put + // TODO:UJ &&& status return??? Probably overkill. + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " " + requestStr + " failed, ex: " + ex.what()); + } + + if (!transmitSuccess) { + // If this fails, wait for + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to send message"); + } +} + +bool WCzarInfo::checkAlive(TIMEPOINT tmMark) { + lock_guard lg(_wciMtx); + if (_alive) { + auto timeSinceContact = tmMark - _lastTouch; + if (timeSinceContact >= 120s) { // TODO:UJ get _deadTime from config &&& + // Contact with the czar has timed out. + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " czar timeout"); + _alive = false; + // Kill all queries from this czar + auto fMan = Foreman::getForeman(); + if (fMan != nullptr) { + auto queriesAndChunks = fMan->getQueriesAndChunks(); + if (queriesAndChunks != nullptr) { + queriesAndChunks->killAllQueriesFromCzar(czarId); + } + } + } + } + return _alive; +} + +WCzarInfo::Ptr WCzarInfoMap::getWCzarInfo(CzarIdType czId) { + std::lock_guard lg(_wczMapMtx); + auto iter = _wczMap.find(czId); + if (iter == _wczMap.end()) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " new czar contacted " << czId); + auto const newCzInfo = WCzarInfo::create(czId); + _wczMap[czId] = newCzInfo; + return newCzInfo; + } + return iter->second; +} + +} // namespace lsst::qserv::wcontrol diff --git a/src/wcontrol/WCzarInfoMap.h b/src/wcontrol/WCzarInfoMap.h new file mode 100644 index 0000000000..46f297daf8 --- /dev/null +++ b/src/wcontrol/WCzarInfoMap.h @@ -0,0 +1,129 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_WCONTROL_WCZARINFOMAP_H +#define LSST_QSERV_WCONTROL_WCZARINFOMAP_H + +// System headers +#include +#include +#include +#include + +// Third-party headers + +// Qserv headers +#include "global/clock_defs.h" +#include "global/intTypes.h" + +namespace lsst::qserv::protojson { +class CzarContactInfo; +class WorkerContactInfo; +class WorkerCzarComIssue; +} // namespace lsst::qserv::protojson + +namespace lsst::qserv::wbase { +class UJTransmitCmd; +} + +namespace lsst::qserv::wcontrol { + +class Foreman; + +/// This class is used to send the "/workerczarcomissue" from the worker to the +/// czar and then used by the czar to handle the message; the messsage itself +/// is made with WorkerCzarComIssue. +/// The general concept is that WorkerCzarComIssue exists on both the worker +/// and the czar and messages keep them in sync. +/// This class is assuming the czardId is correct and there are no duplicate czarIds. +class WCzarInfo : public std::enable_shared_from_this { +public: + using Ptr = std::shared_ptr; + + std::string cName(const char* funcN) { + return std::string("WCzarInfo::") + funcN + " czId=" + std::to_string(czarId); + } + + WCzarInfo() = delete; + ~WCzarInfo() = default; + + static Ptr create(CzarIdType czarId_) { return Ptr(new WCzarInfo(czarId_)); } + + /// If there were communication issues, start a thread to send the WorkerCzarComIssue message. + void sendWorkerCzarComIssueIfNeeded(std::shared_ptr const& wInfo_, + std::shared_ptr const& czInfo_); + + /// Called by the worker after the czar successfully replied to the original + /// message from the worker. + void czarMsgReceived(TIMEPOINT tm); + + bool isAlive() const { return _alive; } + + /// Check if the czar is still considered to be alive, or it timed out. + bool checkAlive(TIMEPOINT tmMark); + + std::shared_ptr getWorkerCzarComIssue(); + + CzarIdType const czarId; + +private: + WCzarInfo(CzarIdType czarId_); + + void _sendMessage(); + + std::atomic _alive{true}; + TIMEPOINT _lastTouch{CLOCK::now()}; + + /// This class tracks communication problems and prepares a message + /// to inform the czar of the problem. + std::shared_ptr _workerCzarComIssue; + mutable std::mutex _wciMtx; ///< protects all private members. + + /// true when running a thread to send a message to the czar + /// with _sendMessage() + std::atomic _msgThreadRunning{false}; +}; + +/// Each worker talks to multiple czars and needs a WCzarInfo object for each czar, +/// this class keeps track of those objects. +class WCzarInfoMap { +public: + using Ptr = std::shared_ptr; + + std::string cName(const char* funcN) { return std::string("WCzarInfoMap::") + funcN; } + + ~WCzarInfoMap() = default; + + static Ptr create() { return Ptr(new WCzarInfoMap()); } + + /// Return the WCzarInfo ptr associated with czId, creating a new one if needed. + WCzarInfo::Ptr getWCzarInfo(CzarIdType czId); + +private: + WCzarInfoMap() = default; + + std::map _wczMap; + + mutable std::mutex _wczMapMtx; +}; + +} // namespace lsst::qserv::wcontrol + +#endif // LSST_QSERV_WCONTROL_WCZARINFOMAP_H diff --git a/src/wcontrol/WorkerStats.cc b/src/wcontrol/WorkerStats.cc index 27055bd046..18a60b6a63 100644 --- a/src/wcontrol/WorkerStats.cc +++ b/src/wcontrol/WorkerStats.cc @@ -45,10 +45,10 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wcontrol.WorkerStats"); namespace lsst::qserv::wcontrol { WorkerStats::Ptr WorkerStats::_globalWorkerStats; -util::Mutex WorkerStats::_globalMtx; +MUTEX WorkerStats::_globalMtx; void WorkerStats::setup() { - lock_guard lg(_globalMtx); + lock_guard lg(_globalMtx); if (_globalWorkerStats != nullptr) { throw util::Bug(ERR_LOC, "Error WorkerStats::setup called after global pointer set."); } @@ -70,7 +70,7 @@ WorkerStats::WorkerStats() { } WorkerStats::Ptr WorkerStats::get() { - std::lock_guard lg(_globalMtx); + std::lock_guard lg(_globalMtx); if (_globalWorkerStats == nullptr) { throw util::Bug(ERR_LOC, "Error CzarStats::get called before CzarStats::setup."); } diff --git a/src/wcontrol/WorkerStats.h b/src/wcontrol/WorkerStats.h index d61f450330..afcde1ed9f 100644 --- a/src/wcontrol/WorkerStats.h +++ b/src/wcontrol/WorkerStats.h @@ -77,7 +77,7 @@ class WorkerStats : std::enable_shared_from_this { private: WorkerStats(); static Ptr _globalWorkerStats; ///< Pointer to the global instance. - static util::Mutex _globalMtx; ///< Protects `_globalWorkerStats` + static MUTEX _globalMtx; ///< Protects `_globalWorkerStats` std::atomic _queueCount{ 0}; ///< Number of buffers on queues (there are many queues, one per ChannelShared) diff --git a/src/wdb/CMakeLists.txt b/src/wdb/CMakeLists.txt index ba92138265..bbf9340312 100644 --- a/src/wdb/CMakeLists.txt +++ b/src/wdb/CMakeLists.txt @@ -4,7 +4,6 @@ add_dependencies(wdb proto) target_sources(wdb PRIVATE ChunkResource.cc QueryRunner.cc - QuerySql.cc SQLBackend.cc ) @@ -40,7 +39,8 @@ ENDFUNCTION() wdb_tests( testChunkResource testQueryRunner - testQuerySql ) +# For this test to work, a mariadb server needs to be available. +# This functionality is covered by integration tests. set_tests_properties(testQueryRunner PROPERTIES WILL_FAIL 1) diff --git a/src/wdb/ChunkResource.cc b/src/wdb/ChunkResource.cc index b131552e29..a9fe100e5b 100644 --- a/src/wdb/ChunkResource.cc +++ b/src/wdb/ChunkResource.cc @@ -48,32 +48,11 @@ #include "util/Bug.h" #include "util/IterableFormatter.h" #include "wbase/Base.h" -#include "wdb/QuerySql.h" namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wdb.ChunkResource"); -template -class ScScriptBuilder { -public: - ScScriptBuilder(lsst::qserv::wdb::QuerySql& qSql_, std::string const& db, std::string const& table, - std::string const& scColumn, int chunkId) - : qSql(qSql_) { - buildT = (boost::format(lsst::qserv::wbase::CREATE_SUBCHUNK_SCRIPT) % db % table % scColumn % - chunkId % "%1%") - .str(); - cleanT = (boost::format(lsst::qserv::wbase::CLEANUP_SUBCHUNK_SCRIPT) % db % table % chunkId % "%1%") - .str(); - } - void operator()(T const& subc) { - qSql.buildList.push_back((boost::format(buildT) % subc).str()); - qSql.cleanupList.push_back((boost::format(cleanT) % subc).str()); - } - std::string buildT; - std::string cleanT; - lsst::qserv::wdb::QuerySql& qSql; -}; } // anonymous namespace namespace lsst::qserv::wdb { diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index 06a2498e8e..33b9a287f4 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -124,7 +124,6 @@ bool QueryRunner::_initConnection() { } bool QueryRunner::runQuery() { - util::InstanceCount ic(to_string(_task->getQueryId()) + "_rq_LDB"); // LockupDB util::HoldTrack::Mark runQueryMarkA(ERR_LOC, "runQuery " + to_string(_task->getQueryId())); QSERV_LOGCONTEXT_QUERY_JOB(_task->getQueryId(), _task->getJobId()); LOGS(_log, LOG_LVL_TRACE, __func__ << " tid=" << _task->getIdStr()); @@ -234,7 +233,7 @@ bool QueryRunner::_dispatchChannel() { // Ideally, hold it until moving on to the next chunk. Try to clean up ChunkResource code. auto taskSched = _task->getTaskScheduler(); - if (!_cancelled && !_task->getSendChannel()->isDead()) { + if (!_cancelled && !_task->checkCancelled()) { string const& query = _task->getQueryString(); util::Timer primeT; primeT.start(); diff --git a/src/wdb/QueryRunner.h b/src/wdb/QueryRunner.h index 91e83db11a..bebe22a208 100644 --- a/src/wdb/QueryRunner.h +++ b/src/wdb/QueryRunner.h @@ -75,8 +75,6 @@ class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_fro /// by Task::cancel(), so if this needs to be cancelled elsewhere, /// call Task::cancel(). /// This should kill an in progress SQL command. - /// It also tries to unblock `_streamBuf` to keep the thread - /// from being blocked forever. void cancel() override; protected: @@ -96,7 +94,6 @@ class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_fro /// Resource reservation ChunkResourceMgr::Ptr _chunkResourceMgr; - std::atomic _cancelled{false}; mysql::MySqlConfig const _mySqlConfig; std::unique_ptr _mysqlConn; diff --git a/src/wdb/QuerySql.cc b/src/wdb/QuerySql.cc deleted file mode 100644 index 7dd1279578..0000000000 --- a/src/wdb/QuerySql.cc +++ /dev/null @@ -1,133 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2012-2015 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -/** - * @file - * - * @brief QuerySql is a bundle of SQL statements that represent an accepted - * query's generated SQL. - * - * FIXME: Unfinished infrastructure for passing subchunk table name to worker. - * - * @author Daniel L. Wang, SLAC - */ - -// Class header -#include "wdb/QuerySql.h" - -// System headers -#include - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "global/constants.h" -#include "global/DbTable.h" -#include "proto/worker.pb.h" -#include "wbase/Base.h" - -namespace { - -LOG_LOGGER _log = LOG_GET("lsst.qserv.wdb.QuerySql"); - -template -class ScScriptBuilder { -public: - ScScriptBuilder(lsst::qserv::wdb::QuerySql& qSql_, std::string const& db, std::string const& table, - std::string const& scColumn, int chunkId) - : qSql(qSql_) { - buildT = (boost::format(lsst::qserv::wbase::CREATE_SUBCHUNK_SCRIPT) % db % table % scColumn % - chunkId % "%1%") - .str(); - cleanT = (boost::format(lsst::qserv::wbase::CLEANUP_SUBCHUNK_SCRIPT) % db % table % chunkId % "%1%") - .str(); - } - void operator()(T const& subc) { - qSql.buildList.push_back((boost::format(buildT) % subc).str()); - qSql.cleanupList.push_back((boost::format(cleanT) % subc).str()); - } - std::string buildT; - std::string cleanT; - lsst::qserv::wdb::QuerySql& qSql; -}; -} // anonymous namespace - -namespace lsst::qserv::wdb { - -//////////////////////////////////////////////////////////////////////// -// QuerySql ostream friend -//////////////////////////////////////////////////////////////////////// -std::ostream& operator<<(std::ostream& os, QuerySql const& q) { - os << "QuerySql(bu="; - std::copy(q.buildList.begin(), q.buildList.end(), std::ostream_iterator(os, ",")); - os << "; ex="; - std::copy(q.executeList.begin(), q.executeList.end(), std::ostream_iterator(os, ",")); - os << "; cl="; - std::copy(q.cleanupList.begin(), q.cleanupList.end(), std::ostream_iterator(os, ",")); - os << ")"; - return os; -} - -//////////////////////////////////////////////////////////////////////// -// QuerySql constructor -//////////////////////////////////////////////////////////////////////// -QuerySql::QuerySql(std::string const& db, int chunkId, proto::TaskMsg_Fragment const& f, bool needCreate, - std::string const& defaultResultTable) { - std::string resultTable; - if (f.has_resulttable()) { - resultTable = f.resulttable(); - } else { - resultTable = defaultResultTable; - } - assert(!resultTable.empty()); - - // Create executable statement. - // Obsolete when results marshalling is implemented - std::stringstream ss; - for (int i = 0; i < f.query_size(); ++i) { - if (needCreate) { - ss << "CREATE TABLE " + resultTable + " "; - needCreate = false; - } else { - ss << "INSERT INTO " + resultTable + " "; - } - ss << f.query(i); - executeList.push_back(ss.str()); - ss.str(""); - } - - if (f.has_subchunks()) { - proto::TaskMsg_Subchunk const& sc = f.subchunks(); - for (int i = 0; i < sc.dbtbl_size(); ++i) { - DbTable dbTable(sc.dbtbl(i).db(), sc.dbtbl(i).tbl()); - LOGS(_log, LOG_LVL_DEBUG, "Building subchunks for table=" << dbTable << " chunkId=" << chunkId); - ScScriptBuilder scb(*this, dbTable.db, dbTable.table, SUB_CHUNK_COLUMN, chunkId); - for (int i = 0; i < sc.id_size(); ++i) { - scb(sc.id(i)); - } - } - } -} - -} // namespace lsst::qserv::wdb diff --git a/src/wdb/QuerySql.h b/src/wdb/QuerySql.h deleted file mode 100644 index cfc2e48bf9..0000000000 --- a/src/wdb/QuerySql.h +++ /dev/null @@ -1,72 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_WDB_QUERYSQL_H -#define LSST_QSERV_WDB_QUERYSQL_H -/** - * @file - * - * @brief QuerySql is a bundle of SQL statements that represent an accepted - * query's generated SQL. - * - * @author Daniel L. Wang, SLAC - */ - -// System headers -#include -#include -#include -#include - -// Forward declarations -namespace lsst::qserv { -namespace proto { -class TaskMsg_Fragment; -} -namespace wdb { -class Task; -} -} // namespace lsst::qserv - -namespace lsst::qserv::wdb { - -class QuerySql { -public: - typedef std::shared_ptr Ptr; - typedef std::deque StringDeque; - typedef lsst::qserv::proto::TaskMsg_Fragment Fragment; - - QuerySql() {} - QuerySql(std::string const& db, int chunkId, proto::TaskMsg_Fragment const& f, bool needCreate, - std::string const& defaultResultTable); - - StringDeque buildList; - StringDeque executeList; // Consider using SqlFragmenter to break this up into fragments. - StringDeque cleanupList; - struct Batch; - friend std::ostream& operator<<(std::ostream& os, QuerySql const& q); -}; - -} // namespace lsst::qserv::wdb - -#endif // LSST_QSERV_WDB_QUERYSQL_H diff --git a/src/wdb/QuerySql_Batch.h b/src/wdb/QuerySql_Batch.h deleted file mode 100644 index bec2783500..0000000000 --- a/src/wdb/QuerySql_Batch.h +++ /dev/null @@ -1,82 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2014 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_WDB_QUERYSQL_BATCH_H -#define LSST_QSERV_WDB_QUERYSQL_BATCH_H -/** - * @file - * - * @brief QuerySql::Batch is the actual bundling portion of a QuerySql object. - * - * @author Daniel L. Wang, SLAC - */ - -// System headers -#include -#include - -// Local headers -#include "wdb/QuerySql.h" - -namespace lsst::qserv::wdb { - -struct QuerySql::Batch { - // Default to 10 SQL statements at a time. - // Idea: Could add statements according to some cost metric(a - // simple one) or to a certain overall query string length - Batch(std::string const& name_, QuerySql::StringDeque const& sequence_, int batchSize_ = 10) - : name(name_), batchSize(batchSize_), pos(0) { - for (QuerySql::StringDeque::const_iterator i = sequence_.begin(); i != sequence_.end(); ++i) { - std::string::const_iterator last = i->begin() + (i->length() - 1); - if (';' == *last) { // Clip trailing semicolon which - // is added during batching. - sequence.push_back(std::string(i->begin(), last)); - } else { - sequence.push_back(*i); - } - } - } - bool isDone() const { return sequence.empty() || (static_cast(pos) >= sequence.size()); } - std::string current() const { - std::ostringstream os; - QuerySql::StringDeque::const_iterator begin; - assert((unsigned)pos < sequence.size()); // caller should have checked isDone() - begin = sequence.begin() + pos; - if (sequence.size() < static_cast(pos + batchSize)) { - std::copy(begin, sequence.end(), std::ostream_iterator(os, ";\n")); - } else { - std::copy(begin, begin + batchSize, std::ostream_iterator(os, ";\n")); - } - return os.str(); - } - void next() { pos += batchSize; } - - std::string name; - QuerySql::StringDeque sequence; - QuerySql::StringDeque::size_type batchSize; - QuerySql::StringDeque::size_type pos; -}; - -} // namespace lsst::qserv::wdb - -#endif // LSST_QSERV_WDB_QUERYSQL_BATCH_H diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 319d4252b2..c67acf74a2 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -29,10 +29,13 @@ // Qserv headers #include "mysql/MySqlConfig.h" +#include "protojson/ScanTableInfo.h" #include "proto/worker.pb.h" #include "wbase/FileChannelShared.h" #include "wbase/Task.h" +#include "wbase/UberJobData.h" #include "wconfig/WorkerConfig.h" +#include "wcontrol/Foreman.h" #include "wcontrol/SqlConnMgr.h" #include "wdb/ChunkResource.h" #include "wdb/QueryRunner.h" @@ -51,10 +54,6 @@ namespace util = lsst::qserv::util; using lsst::qserv::mysql::MySqlConfig; using lsst::qserv::mysql::MySqlConnection; -using lsst::qserv::proto::TaskMsg; -using lsst::qserv::proto::TaskMsg_Fragment; -using lsst::qserv::proto::TaskMsg_Subchunk; - using lsst::qserv::wbase::FileChannelShared; using lsst::qserv::wbase::SendChannel; using lsst::qserv::wbase::Task; @@ -67,18 +66,68 @@ using lsst::qserv::wdb::QueryRunner; using lsst::qserv::wpublish::QueriesAndChunks; struct Fixture { - shared_ptr newTaskMsg() { - shared_ptr t = make_shared(); - t->set_chunkid(3240); // hardcoded - t->set_db("LSST"); // hardcoded - auto scanTbl = t->add_scantable(); - scanTbl->set_db("LSST"); - scanTbl->set_table("Object"); - scanTbl->set_lockinmemory(false); - scanTbl->set_scanrating(1); - lsst::qserv::proto::TaskMsg::Fragment* f = t->add_fragment(); - f->add_query("SELECT AVG(yFlux_PS) from LSST.Object_3240"); - return t; + struct MsgInfo { + string const db = "LSST"; + string const table = "Object"; + string const qry = "SELECT AVG(yFlux_PS) from LSST.Object_3240"; + int const chunkId = 3240; + int const czarId = 5; + string const czarName = "cz5"; + string const czarHostName = "cz5host"; + int const czarPort = 3437; + string const targWorkerId = "a_worker"; + std::shared_ptr foreman; + int const queryId = 23; + int const jobId = 1; + int const uberJobId = 1; + int const attemptCount = 1; + int const scanRating = 1; + bool const scanInteractive = false; + int const maxTableSize = 5000; + bool const lockInMemory = false; + string const resultName = "resName"; + string const authKey = "noAuthKey"; + int const rowLimit = 0; + }; + + shared_ptr newTaskJson(MsgInfo const& mInfo) { + // Derived from TaskMsgFactory::makeMsgJson + + auto jsJobMsgPtr = std::shared_ptr( + new nlohmann::json({{"czarId", mInfo.czarId}, + {"queryId", mInfo.queryId}, + {"jobId", mInfo.jobId}, + {"attemptCount", mInfo.attemptCount}, + {"querySpecDb", mInfo.db}, + {"scanPriority", mInfo.scanRating}, + {"scanInteractive", mInfo.scanInteractive}, + {"maxTableSize", mInfo.maxTableSize}, + {"chunkScanTables", nlohmann::json::array()}, + {"chunkId", mInfo.chunkId}, + {"queryFragments", nlohmann::json::array()}})); + + auto& jsJobMsg = *jsJobMsgPtr; + + auto& chunkScanTables = jsJobMsg["chunkScanTables"]; + nlohmann::json cst = {{"db", mInfo.db}, + {"table", mInfo.table}, + {"lockInMemory", mInfo.lockInMemory}, + {"tblScanRating", mInfo.scanRating}}; + chunkScanTables.push_back(move(cst)); + + auto& jsFragments = jsJobMsg["queryFragments"]; + nlohmann::json jsFrag = {{"resultTable", mInfo.resultName}, + {"queries", nlohmann::json::array()}, + {"subchunkTables", nlohmann::json::array()}, + {"subchunkIds", nlohmann::json::array()}}; + + auto& jsQueries = jsFrag["queries"]; + nlohmann::json jsQry = {{"subQuery", mInfo.qry}}; + jsQueries.push_back(move(jsQry)); + + jsFragments.push_back(move(jsFrag)); + + return jsJobMsgPtr; } MySqlConfig newMySqlConfig() { @@ -100,18 +149,28 @@ struct Fixture { } }; -BOOST_FIXTURE_TEST_SUITE(Basic, Fixture) +BOOST_FIXTURE_TEST_SUITE(Basic, Fixture, *boost::unit_test::timeout(20)) BOOST_AUTO_TEST_CASE(Simple) { WorkerConfig::create(); - shared_ptr msg(newTaskMsg()); + MsgInfo mInfo; + auto msgJson = newTaskJson(mInfo); shared_ptr sendC(SendChannel::newNopChannel()); - auto sc = FileChannelShared::create(sendC, msg->czarid()); + auto sChannel = FileChannelShared::create(sendC, mInfo.czarId); FakeBackend::Ptr backend = make_shared(); shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); + SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto taskVect = Task::createTasks(msg, sc, crm, newMySqlConfig(), sqlConnMgr, queries); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); + scanInfo->scanRating = mInfo.scanRating; + scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + vector taskVect = + Task::createTasksForUnitTest(ujData, *msgJson, sChannel, scanInfo, mInfo.scanInteractive, + mInfo.maxTableSize, crm); + Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); @@ -120,14 +179,24 @@ BOOST_AUTO_TEST_CASE(Simple) { BOOST_AUTO_TEST_CASE(Output) { WorkerConfig::create(); string out; - shared_ptr msg(newTaskMsg()); + MsgInfo mInfo; + auto msgJson = newTaskJson(mInfo); shared_ptr sendC(SendChannel::newStringChannel(out)); - auto sc = FileChannelShared::create(sendC, msg->czarid()); + auto sc = FileChannelShared::create(sendC, mInfo.czarId); FakeBackend::Ptr backend = make_shared(); shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); + SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto taskVect = Task::createTasks(msg, sc, crm, newMySqlConfig(), sqlConnMgr, queries); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); + scanInfo->scanRating = mInfo.scanRating; + scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + vector taskVect = + Task::createTasksForUnitTest(ujData, *msgJson, sc, scanInfo, mInfo.scanInteractive, + mInfo.maxTableSize, crm); + Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); diff --git a/src/wdb/testQuerySql.cc b/src/wdb/testQuerySql.cc deleted file mode 100644 index 5d7cd46071..0000000000 --- a/src/wdb/testQuerySql.cc +++ /dev/null @@ -1,99 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2015 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/** - * @brief Simple testing for class QuerySql - * - * @author Daniel L. Wang, SLAC - */ - -// Third-party headers - -// Qserv headers -#include "proto/worker.pb.h" -#include "wdb/QuerySql.h" -#include "wdb/QuerySql_Batch.h" - -// Boost unit test header -#define BOOST_TEST_MODULE QuerySql_1 -#include - -namespace test = boost::test_tools; - -using lsst::qserv::proto::TaskMsg_Fragment; -using lsst::qserv::proto::TaskMsg_Subchunk; -using lsst::qserv::wdb::QuerySql; - -struct Fixture { - Fixture() { - defaultDb = "Winter"; - defaultResult = "myResult"; - } - ~Fixture() {} - - TaskMsg_Fragment makeFragment() { - TaskMsg_Fragment f; - // "Real" subchunk query text should include - // pre-substituted subchunk query text. - f.add_query("SELECT o1.*, o2.* FROM Object_1001 o1, Object_1001 o2;"); - f.set_resulttable("fragResult"); - TaskMsg_Subchunk sc; - sc.set_database("obsolete"); - lsst::qserv::proto::TaskMsg_Subchunk_DbTbl* dbTbl = sc.add_dbtbl(); - dbTbl->set_db(defaultDb); - dbTbl->set_tbl("Object"); - sc.add_id(1111); - sc.add_id(1222); - f.mutable_subchunks()->CopyFrom(sc); - return f; - } - - void printQsql(QuerySql const& q) { std::cout << "qsql=" << q << std::endl; } - std::string defaultDb; - std::string defaultResult; -}; - -BOOST_FIXTURE_TEST_SUITE(QuerySqlSuite, Fixture) - -BOOST_AUTO_TEST_CASE(Basic) { - std::shared_ptr qSql; - TaskMsg_Fragment frag = makeFragment(); - qSql = std::make_shared(defaultDb, 1001, frag, true, defaultResult); - BOOST_CHECK(qSql.get()); - printQsql(*qSql); -} - -BOOST_AUTO_TEST_CASE(QueryBatch) { - std::shared_ptr qSql; - TaskMsg_Fragment frag = makeFragment(); - qSql = std::make_shared(defaultDb, 1001, frag, true, defaultResult); - BOOST_CHECK(qSql.get()); - - QuerySql::Batch build("QueryBuildSub", qSql->buildList); - QuerySql::Batch& batch = build; - while (!batch.isDone()) { - std::string piece = batch.current(); - batch.next(); - } -} - -BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wpublish/ChunkInventory.cc b/src/wpublish/ChunkInventory.cc index eb112303ab..8bf735602d 100644 --- a/src/wpublish/ChunkInventory.cc +++ b/src/wpublish/ChunkInventory.cc @@ -133,22 +133,6 @@ void fetchId(string const& instanceName, SqlConnection& sc, string& id) { LOGS(_log, LOG_LVL_WARN, "ChunkInventory couldn't find any a unique identifier of the worker"); } -class Validator : public lsst::qserv::ResourceUnit::Checker { -public: - Validator(lsst::qserv::wpublish::ChunkInventory& c) : chunkInventory(c) {} - virtual bool operator()(lsst::qserv::ResourceUnit const& ru) { - switch (ru.unitType()) { - case lsst::qserv::ResourceUnit::DBCHUNK: - return chunkInventory.has(ru.db(), ru.chunk()); - case lsst::qserv::ResourceUnit::QUERY: - return true; - default: - return false; - } - } - lsst::qserv::wpublish::ChunkInventory& chunkInventory; -}; - } // anonymous namespace namespace lsst::qserv::wpublish { @@ -284,10 +268,6 @@ bool ChunkInventory::has(string const& db, int chunk) const { return true; } -shared_ptr ChunkInventory::newValidator() { - return shared_ptr(new Validator(*this)); -} - void ChunkInventory::dbgPrint(ostream& os) const { lock_guard lock(_mtx); diff --git a/src/wpublish/QueriesAndChunks.cc b/src/wpublish/QueriesAndChunks.cc index 6381d08260..1dd21fc791 100644 --- a/src/wpublish/QueriesAndChunks.cc +++ b/src/wpublish/QueriesAndChunks.cc @@ -119,21 +119,25 @@ void QueriesAndChunks::setBlendScheduler(shared_ptr cons void QueriesAndChunks::setRequiredTasksCompleted(unsigned int value) { _requiredTasksCompleted = value; } -/// Add statistics for the Task, creating a QueryStatistics object if needed. -void QueriesAndChunks::addTask(wbase::Task::Ptr const& task) { - auto qid = task->getQueryId(); +QueryStatistics::Ptr QueriesAndChunks::addQueryId(QueryId qId, CzarIdType czarId) { unique_lock guardStats(_queryStatsMapMtx); - auto itr = _queryStatsMap.find(qid); + auto itr = _queryStatsMap.find(qId); QueryStatistics::Ptr stats; if (_queryStatsMap.end() == itr) { - stats = QueryStatistics::create(qid); - _queryStatsMap[qid] = stats; + stats = QueryStatistics::create(qId, czarId); + _queryStatsMap[qId] = stats; } else { stats = itr->second; } - guardStats.unlock(); + return stats; +} + +/// Add statistics for the Task, creating a QueryStatistics object if needed. +void QueriesAndChunks::addTask(wbase::Task::Ptr const& task) { + auto qid = task->getQueryId(); + auto czId = task->getCzarId(); + auto stats = addQueryId(qid, czId); stats->addTask(task); - task->setQueryStatistics(stats); } /// Update statistics for the Task that was just queued. @@ -195,10 +199,11 @@ void QueriesAndChunks::_finishedTaskForChunk(wbase::Task::Ptr const& task, doubl } auto ptr = res.first->second; ul.unlock(); - proto::ScanInfo& scanInfo = task->getScanInfo(); + auto iter = res.first->second; + protojson::ScanInfo::Ptr scanInfo = task->getScanInfo(); string tblName; - if (!scanInfo.infoTables.empty()) { - proto::ScanTableInfo& sti = scanInfo.infoTables.at(0); + if (!scanInfo->infoTables.empty()) { + protojson::ScanTableInfo& sti = scanInfo->infoTables.at(0); tblName = ChunkTableStats::makeTableName(sti.db, sti.table); } ChunkTableStats::Ptr tableStats = ptr->add(tblName, minutes); @@ -257,7 +262,7 @@ void QueriesAndChunks::removeDead(QueryStatistics::Ptr const& queryStats) { _queryStatsMap.erase(qId); } -QueryStatistics::Ptr QueriesAndChunks::getStats(QueryId const& qId) const { +QueryStatistics::Ptr QueriesAndChunks::getStats(QueryId qId) const { lock_guard lockG(_queryStatsMapMtx); return _getStats(qId); } @@ -324,8 +329,8 @@ void QueriesAndChunks::examineAll() { } double schedMaxTime = sched->getMaxTimeMinutes(); // Get max time for scheduler // Get the slowest scan table in task. - auto begin = task->getScanInfo().infoTables.begin(); - if (begin == task->getScanInfo().infoTables.end()) { + auto begin = task->getScanInfo()->infoTables.begin(); + if (begin == task->getScanInfo()->infoTables.end()) { continue; } string const& slowestTable = begin->db + ":" + begin->table; @@ -672,6 +677,23 @@ vector QueriesAndChunks::removeQueryFrom(QueryId const& qId, return removedList; } +void QueriesAndChunks::killAllQueriesFromCzar(CzarIdType czarId) { + std::map qsMap; + { + lock_guard lgQsm(_queryStatsMapMtx); + qsMap = _queryStatsMap; + } + + for (auto const& [qsKey, qsPtr] : qsMap) { + if (qsPtr != nullptr) { + auto uqInfo = qsPtr->getUserQueryInfo(); + if (uqInfo != nullptr && uqInfo->getCzarId() == czarId) { + uqInfo->cancelAllUberJobs(); + } + } + } +} + ostream& operator<<(ostream& os, QueriesAndChunks const& qc) { lock_guard g(qc._chunkMtx); os << "Chunks("; diff --git a/src/wpublish/QueriesAndChunks.h b/src/wpublish/QueriesAndChunks.h index a51e1d24d2..b89458ba3c 100644 --- a/src/wpublish/QueriesAndChunks.h +++ b/src/wpublish/QueriesAndChunks.h @@ -193,8 +193,18 @@ class QueriesAndChunks { void removeDead(); void removeDead(QueryStatistics::Ptr const& queryStats); - /// Return the statistics for a user query. - QueryStatistics::Ptr getStats(QueryId const& qId) const; + /// Return the statistics for a user query, may be nullptr, + /// in many cases addQueryId() may be preferable if + /// new information is being added to the returned object. + /// @see addQueryId() + QueryStatistics::Ptr getStats(QueryId qId) const; + + /// Return the statistics for a user query, creating if needed. + /// Since it is possible to get messages out of order, there + /// are several case where something like a cancellation + /// message arrives before any tasks have been created. + /// @see getStats() + QueryStatistics::Ptr addQueryId(QueryId qId, CzarIdType czarId); void addTask(wbase::Task::Ptr const& task); void queuedTask(wbase::Task::Ptr const& task); @@ -234,6 +244,10 @@ class QueriesAndChunks { }; using ScanTableSumsMap = std::map; + /// If the worker believes this czar has died, it calls this to stop + /// all Tasks associated with that czar. + void killAllQueriesFromCzar(CzarIdType czarId); + friend std::ostream& operator<<(std::ostream& os, QueriesAndChunks const& qc); private: diff --git a/src/wpublish/QueryStatistics.cc b/src/wpublish/QueryStatistics.cc index 576effdee2..2ca96d7f37 100644 --- a/src/wpublish/QueryStatistics.cc +++ b/src/wpublish/QueryStatistics.cc @@ -50,7 +50,10 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wpublish.QueriesAndChunks"); namespace lsst::qserv::wpublish { -QueryStatistics::QueryStatistics(QueryId const& qId_) : creationTime(CLOCK::now()), queryId(qId_) { +QueryStatistics::QueryStatistics(QueryId qId_, CzarIdType czarId_) + : creationTime(CLOCK::now()), + queryId(qId_), + _userQueryInfo(wbase::UserQueryInfo::create(qId_, czarId_)) { /// For all of the histograms, all entries should be kept at least until the work is finished. string qidStr = to_string(queryId); _histSizePerTask = util::Histogram::Ptr(new util::Histogram( diff --git a/src/wpublish/QueryStatistics.h b/src/wpublish/QueryStatistics.h index dc26a9da4c..5fd24ff003 100644 --- a/src/wpublish/QueryStatistics.h +++ b/src/wpublish/QueryStatistics.h @@ -43,8 +43,8 @@ #include "wsched/SchedulerBase.h" namespace lsst::qserv::wbase { -class Histogram; -} +class UserQueryInfo; +} // namespace lsst::qserv::wbase // This header declarations namespace lsst::qserv::wpublish { @@ -56,8 +56,8 @@ class QueryStatistics { using Ptr = std::shared_ptr; /// Force shared_ptr creation for data integrity. - static Ptr create(QueryId const& queryId) { - return std::shared_ptr(new QueryStatistics(queryId)); + static Ptr create(QueryId queryId_, CzarIdType czarId_) { + return std::shared_ptr(new QueryStatistics(queryId_, czarId_)); } QueryStatistics() = delete; @@ -73,6 +73,8 @@ class QueryStatistics { return _queryBooted; } + std::shared_ptr getUserQueryInfo() const { return _userQueryInfo; } + void setQueryBooted(bool booted, TIMEPOINT now); /// Add statistics related to the running of the query in the task. @@ -167,7 +169,7 @@ class QueryStatistics { friend std::ostream& operator<<(std::ostream& os, QueryStatistics const& q); private: - explicit QueryStatistics(QueryId const& queryId); + explicit QueryStatistics(QueryId queryId, CzarIdType czarId); bool _isMostlyDead() const; mutable std::mutex _qStatsMtx; @@ -194,6 +196,9 @@ class QueryStatistics { std::shared_ptr _histRowsPerTask; ///< Histogram of rows per Task. SchedTasksInfoMap _taskSchedInfoMap; ///< Map of task information ordered by scheduler name. + + /// Contains information common to all Tasks in this user query. + std::shared_ptr const _userQueryInfo; }; } // namespace lsst::qserv::wpublish diff --git a/src/wsched/BlendScheduler.cc b/src/wsched/BlendScheduler.cc index 3e9babc06a..b5b37346f4 100644 --- a/src/wsched/BlendScheduler.cc +++ b/src/wsched/BlendScheduler.cc @@ -175,7 +175,7 @@ void BlendScheduler::queCmd(std::vector const& cmds) { if (first) { first = false; - auto const& scanTables = task->getScanInfo().infoTables; + auto const& scanTables = task->getScanInfo()->infoTables; bool interactive = task->getScanInteractive(); if (scanTables.size() <= 0 || interactive) { // If there are no scan tables, no point in putting on a shared scan. @@ -186,7 +186,7 @@ void BlendScheduler::queCmd(std::vector const& cmds) { targSched = _group; } else { onInteractive = false; - int scanPriority = task->getScanInfo().scanRating; + int scanPriority = task->getScanInfo()->scanRating; if (LOG_CHECK_LVL(_log, LOG_LVL_DEBUG)) { ostringstream ss; ss << "Blend chose scan for priority=" << scanPriority << " : "; @@ -259,6 +259,7 @@ void BlendScheduler::commandStart(util::Command::Ptr const& cmd) { LOGS(_log, LOG_LVL_ERROR, "BlendScheduler::commandStart scheduler not found"); } _infoChanged = true; + LOGS(_log, LOG_LVL_DEBUG, "BlendScheduler::commandStart &&& end"); } void BlendScheduler::commandFinish(util::Command::Ptr const& cmd) { diff --git a/src/wsched/ChunkTasksQueue.h b/src/wsched/ChunkTasksQueue.h index b485ee0e12..baa48af756 100644 --- a/src/wsched/ChunkTasksQueue.h +++ b/src/wsched/ChunkTasksQueue.h @@ -82,7 +82,7 @@ class ChunkTasks { return false; } // compare scanInfo (slower scans first) - int siComp = x->getScanInfo().compareTables(y->getScanInfo()); + int siComp = x->getScanInfo()->compareTables(*(y->getScanInfo())); return siComp < 0; }; void push(wbase::Task::Ptr const& task); diff --git a/src/wsched/GroupScheduler.cc b/src/wsched/GroupScheduler.cc index 5b5c7da270..2429f7ee3b 100644 --- a/src/wsched/GroupScheduler.cc +++ b/src/wsched/GroupScheduler.cc @@ -116,7 +116,7 @@ void GroupScheduler::_queCmd(util::Command::Ptr const& cmd, bool keepInThisGroup } auto uqCount = _incrCountForUserQuery(t->getQueryId(), 1); LOGS(_log, LOG_LVL_DEBUG, - getName() << " queCmd uqCount=" << uqCount << " rating=" << t->getScanInfo().scanRating + getName() << " queCmd uqCount=" << uqCount << " rating=" << t->getScanInfo()->scanRating << " interactive=" << t->getScanInteractive()); util::CommandQueue::_cv.notify_one(); } diff --git a/src/wsched/testSchedulers.cc b/src/wsched/testSchedulers.cc index f27b7269c9..f2217b1b68 100644 --- a/src/wsched/testSchedulers.cc +++ b/src/wsched/testSchedulers.cc @@ -32,8 +32,8 @@ // Qserv headers #include "mysql/MySqlConfig.h" -#include "proto/ScanTableInfo.h" #include "proto/worker.pb.h" +#include "protojson/ScanTableInfo.h" #include "util/Command.h" #include "util/EventThread.h" #include "wbase/FileChannelShared.h" @@ -83,6 +83,7 @@ auto workerCfg = lsst::qserv::wconfig::WorkerConfig::create(); std::vector locSendSharedPtrs; +/* &&& Task::Ptr makeTask(std::shared_ptr tm, shared_ptr const& queries) { WorkerConfig::create(); auto sendC = std::make_shared(); @@ -92,6 +93,7 @@ Task::Ptr makeTask(std::shared_ptr tm, shared_ptr con Task::Ptr task = taskVect[0]; return task; } +*/ struct SchedulerFixture { typedef std::shared_ptr TaskMsgPtr; @@ -99,6 +101,7 @@ struct SchedulerFixture { SchedulerFixture(void) { counter = 20; } ~SchedulerFixture(void) {} + /* &&& Instead of using messages, make a Task::createTasksForUnitTest() function void addSomeFragments(TaskMsgPtr const& t, int numberOfFragments) { for (int i = 0; i < numberOfFragments; ++i) { TaskMsg::Fragment* f = t->add_fragment(); @@ -108,6 +111,7 @@ struct SchedulerFixture { } } + TaskMsgPtr newTaskMsg(int seq, lsst::qserv::QueryId qId, int jobId) { TaskMsgPtr t = std::make_shared(); t->set_queryid(qId); @@ -115,7 +119,7 @@ struct SchedulerFixture { t->set_chunkid(seq); t->set_czarid(1); t->set_db("elephant"); - addSomeFragments(t, 3); + //&&&addSomeFragments(t, 3); t->set_scaninteractive(false); t->set_attemptcount(0); ++counter; @@ -131,7 +135,7 @@ struct SchedulerFixture { t->set_db("moose"); t->set_scaninteractive(false); t->set_attemptcount(0); - addSomeFragments(t, 1); + //&&&addSomeFragments(t, 1); ++counter; return t; } @@ -154,6 +158,7 @@ struct SchedulerFixture { gs.queCmd(t); return t; } + */ int counter; }; @@ -181,10 +186,10 @@ struct SchedFixture { queries->setRequiredTasksCompleted(1); // Make it easy to set a baseline. } - int const fastest = lsst::qserv::proto::ScanInfo::Rating::FASTEST; - int const fast = lsst::qserv::proto::ScanInfo::Rating::FAST; - int const medium = lsst::qserv::proto::ScanInfo::Rating::MEDIUM; - int const slow = lsst::qserv::proto::ScanInfo::Rating::SLOW; + int const fastest = lsst::qserv::protojson::ScanInfo::Rating::FASTEST; + int const fast = lsst::qserv::protojson::ScanInfo::Rating::FAST; + int const medium = lsst::qserv::protojson::ScanInfo::Rating::MEDIUM; + int const slow = lsst::qserv::protojson::ScanInfo::Rating::SLOW; lsst::qserv::QueryId qIdInc{1}; @@ -213,6 +218,7 @@ struct SchedFixture { // TODO: DM-33302 replace this test case BOOST_AUTO_TEST_CASE(Grouping) { +#if 0 // &&& fix and re-enable SchedFixture f(60.0, 1); // Values to keep QueriesAndChunk from triggering. LOGS(_log, LOG_LVL_DEBUG, "Test_case grouping"); @@ -293,9 +299,11 @@ BOOST_AUTO_TEST_CASE(Grouping) { BOOST_CHECK(gs.getInFlight() == 10); BOOST_CHECK(gs.ready() == false); BOOST_CHECK(gs.empty() == true); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(GroupMaxThread) { +#if 0 // &&& fix and re-enable // Test that maxThreads is meaningful. LOGS(_log, LOG_LVL_WARN, "Test_case GroupMaxThread"); auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, @@ -326,9 +334,11 @@ BOOST_AUTO_TEST_CASE(GroupMaxThread) { auto aa4 = gs.getCmd(false); BOOST_CHECK(a4.get() == aa4.get()); BOOST_CHECK(gs.ready() == false); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(ScanScheduleTest) { +#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case ScanScheduleTest"); auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, maxDarkTasksC, resetForTestingC); @@ -390,9 +400,11 @@ BOOST_AUTO_TEST_CASE(ScanScheduleTest) { sched.commandFinish(tsk1); BOOST_CHECK(sched.getInFlight() == 0); BOOST_CHECK(sched.ready() == false); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleTest) { +#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case BlendScheduleTest"); // Test that space is appropriately reserved for each scheduler as Tasks are started and finished. // TODO: This needs to be evaluated and removed. @@ -592,9 +604,11 @@ BOOST_AUTO_TEST_CASE(BlendScheduleTest) { BOOST_CHECK(f.blend->calcAvailableTheads() == 5); BOOST_CHECK(f.blend->getInFlight() == 0); LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-1 done"); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleThreadLimitingTest) { +#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case BlendScheduleThreadLimitingTest"); SchedFixture f(60.0, 1); // Values to keep QueriesAndChunk from triggering. // Test that only 6 threads can be started on a single ScanScheduler @@ -662,9 +676,11 @@ BOOST_AUTO_TEST_CASE(BlendScheduleThreadLimitingTest) { BOOST_CHECK(f.blend->getInFlight() == 0); BOOST_CHECK(f.blend->ready() == false); LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-2 done"); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleQueryRemovalTest) { +#if 0 // &&& fix and re-enable // Test that space is appropriately reserved for each scheduler as Tasks are started and finished. // TODO: This needs to be evaluated and removed. // In this case, memMan->lock(..) always returns true (really HandleType::ISEMPTY). @@ -723,9 +739,11 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryRemovalTest) { auto schedForA = std::dynamic_pointer_cast(taskFromA->getTaskScheduler()); LOGS(_log, LOG_LVL_DEBUG, "taskFromA=" << taskFromA->getIdStr() << " sched=" << schedForA->getName()); BOOST_CHECK(schedForA == f.scanSlow); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleQueryBootTaskTest) { +#if 0 // &&& fix and re-enable // Test if a task is removed if it takes takes too long. // Give the user query 0.1 seconds to run and run it for a second, it should get removed. double tenthOfSecInMinutes = 1.0 / 600.0; // task @@ -807,9 +825,11 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryBootTaskTest) { LOGS(_log, LOG_LVL_INFO, "BlendScheduleQueryBootTaskTest waiting for pool to finish."); pool->shutdownPool(); LOGS(_log, LOG_LVL_INFO, "BlendScheduleQueryBootTaskTest done"); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(SlowTableHeapTest) { +#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case SlowTableHeapTest start"); auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, maxDarkTasksC, resetForTestingC); @@ -842,9 +862,11 @@ BOOST_AUTO_TEST_CASE(SlowTableHeapTest) { BOOST_CHECK(heap.pop().get() == a4.get()); BOOST_CHECK(heap.empty() == true); LOGS(_log, LOG_LVL_DEBUG, "SlowTableHeapTest done"); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(ChunkTasksTest) { +#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case ChunkTasksTest start"); auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, maxDarkTasksC, resetForTestingC); @@ -913,9 +935,11 @@ BOOST_AUTO_TEST_CASE(ChunkTasksTest) { chunkTasks.taskComplete(a4); BOOST_CHECK(chunkTasks.readyToAdvance() == true); LOGS(_log, LOG_LVL_DEBUG, "ChunkTasksTest done"); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(ChunkTasksQueueTest) { +#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case ChunkTasksQueueTest start"); auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, maxDarkTasksC, resetForTestingC); @@ -1029,6 +1053,7 @@ BOOST_AUTO_TEST_CASE(ChunkTasksQueueTest) { BOOST_CHECK(ctl.ready(true) == false); BOOST_CHECK(ctl.getActiveChunkId() == -1); LOGS(_log, LOG_LVL_DEBUG, "ChunkTasksQueueTest done"); +#endif // &&& fix and re-enable } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/xrdreq/CMakeLists.txt b/src/xrdreq/CMakeLists.txt deleted file mode 100644 index f71141ccc7..0000000000 --- a/src/xrdreq/CMakeLists.txt +++ /dev/null @@ -1,49 +0,0 @@ -add_library(xrdreq SHARED) -add_dependencies(xrdreq proto) - -target_sources(xrdreq PRIVATE - QservRequest.cc - QueryManagementAction.cc - QueryManagementRequest.cc -) - -target_include_directories(xrdreq PRIVATE - ${XROOTD_INCLUDE_DIRS} -) - -target_link_libraries(xrdreq PUBLIC - log - proto - protobuf - XrdSsiLib - XrdCl -) - -install( - TARGETS xrdreq -) - -FUNCTION(XRDREQ_UTILS) - FOREACH(UTIL IN ITEMS ${ARGV}) - add_executable(${UTIL}) - target_sources(${UTIL} PRIVATE ${UTIL}.cc) - target_include_directories(${UTIL} PRIVATE ${XROOTD_INCLUDE_DIRS}) - target_link_libraries(${UTIL} PRIVATE - crypto - pthread - proto - util - global - xrdreq - ) - install(TARGETS ${UTIL}) - ENDFOREACH() -ENDFUNCTION() - -xrdreq_utils( - qserv-query-management -) - -install( - TARGETS xrdreq -) diff --git a/src/xrdreq/QservRequest.cc b/src/xrdreq/QservRequest.cc deleted file mode 100644 index 6310d1c096..0000000000 --- a/src/xrdreq/QservRequest.cc +++ /dev/null @@ -1,216 +0,0 @@ -/* - * LSST Data Management System - * Copyright 2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "xrdreq/QservRequest.h" - -// System headers -#include -#include - -// Qserv headers -#include "lsst/log/Log.h" - -using namespace std; - -namespace { - -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdreq.QservRequest"); - -// Set this parameter to some reasonable default -int const bufInitialSize = 1024; - -} // namespace - -namespace lsst::qserv::xrdreq { - -atomic QservRequest::_numClassInstances(0); - -QservRequest::~QservRequest() { - delete[] _buf; - - --_numClassInstances; - LOGS(_log, LOG_LVL_TRACE, "QservRequest destructed instances: " << _numClassInstances); -} - -QservRequest::QservRequest() - : _bufIncrementSize(bufInitialSize), - _bufSize(0), - _bufCapacity(bufInitialSize), - _buf(new char[bufInitialSize]) { - // This report is used solely for debugging purposes to allow tracking - // potential memory leaks within applications. - ++_numClassInstances; - LOGS(_log, LOG_LVL_TRACE, "QservRequest constructed instances: " << _numClassInstances); -} - -void QservRequest::cancel() { - // This will decrement the reference counter to the pointee at the end of the current - // block regardless of any exceptions that may be thrown below. - auto self = move(_refToSelf4keepAlive); - Finished(true); -} - -void QservRequest::setRefToSelf4keepAlive(shared_ptr ptr) { - if ((ptr == nullptr) || (this != ptr.get())) { - stringstream ss; - ss << "QservRequest::" << __func__ << ": the value of " << ptr - << " passed as an argument is not pointing to the current object."; - throw invalid_argument(ss.str()); - } - _refToSelf4keepAlive = ptr; -} - -char* QservRequest::GetRequest(int& dlen) { - // Ask a subclass to serialize its request into the frame buffer - onRequest(_frameBuf); - - // Tell SSI which data and how many bytes to send - dlen = _frameBuf.size(); - return _frameBuf.data(); -} - -bool QservRequest::ProcessResponse(const XrdSsiErrInfo& eInfo, const XrdSsiRespInfo& rInfo) { - string const context = "QservRequest::" + string(__func__) + " "; - - if (eInfo.hasError()) { - // This will decrement the reference counter to the pointee at the end of the current - // block regardless of any exceptions that may be thrown below. - auto self = move(_refToSelf4keepAlive); - - // Copy the argument before sending the upstream notification - // Otherwise the current object may get disposed before we even had - // a chance to notify XRootD/SSI by calling Finished(). - string const errorStr = rInfo.eMsg; - - LOGS(_log, LOG_LVL_ERROR, context << "** FAILED **, error: " << errorStr); - - // Tell XrootD to release all resources associated with this request - Finished(); - - // Notify a subclass on the abnormal condition - // WARNING: This has to be the last call as the object may get deleted - // downstream. - onError(errorStr); - return false; - } - LOGS(_log, LOG_LVL_TRACE, - context << " eInfo.rType: " << rInfo.rType << "(" << rInfo.State() << ")" - << ", eInfo.blen: " << rInfo.blen); - - switch (rInfo.rType) { - case XrdSsiRespInfo::isData: - case XrdSsiRespInfo::isStream: - - LOGS(_log, LOG_LVL_TRACE, context << "** REQUESTING RESPONSE DATA **"); - GetResponseData(_buf + _bufSize, _bufIncrementSize); - return true; - - default: - // This will decrement the reference counter to the pointee at the end of the current - // block regardless of any exceptions that may be thrown below. - auto self = move(_refToSelf4keepAlive); - - // Copy the argument before sending the upstream notification - // Otherwise the current object may get disposed before we even had - // a chance to notify XRootD/SSI by calling Finished(). - string const responseType = to_string(rInfo.rType); - - // Tell XrootD to release all resources associated with this request - Finished(); - - // Notify a subclass on the abnormal condition - // WARNING: This has to be the last call as the object may get deleted - // downstream. - onError("QservRequest::ProcessResponse ** ERROR ** unexpected response type: " + responseType); - return false; - } -} - -void QservRequest::ProcessResponseData(const XrdSsiErrInfo& eInfo, char* buff, int blen, bool last) { - string const context = "QservRequest::" + string(__func__) + " "; - - LOGS(_log, LOG_LVL_TRACE, context << "eInfo.isOK: " << eInfo.isOK()); - - if (not eInfo.isOK()) { - // This will decrement the reference counter to the pointee at the end of the current - // block regardless of any exceptions that may be thrown below. - auto self = move(_refToSelf4keepAlive); - - // Copy these arguments before sending the upstream notification. - // Otherwise the current object may get disposed before we even had - // a chance to notify XRootD/SSI by calling Finished(). - - string const errorStr = eInfo.Get(); - int const errorNum = eInfo.GetArg(); - - LOGS(_log, LOG_LVL_ERROR, - context << "** FAILED ** eInfo.Get(): " << errorStr << ", eInfo.GetArg(): " << errorNum); - - // Tell XrootD to realease all resources associated with this request - Finished(); - - // Notify a subclass on the ubnormal condition. - // WARNING: This has to be the last call as the object may get deleted - // downstream. - onError(errorStr); - - } else { - LOGS(_log, LOG_LVL_TRACE, context << "blen: " << blen << ", last: " << last); - - // Update the byte counter - _bufSize += blen; - - if (last) { - // This will decrement the reference counter to the pointee at the end of the current - // block regardless of any exceptions that may be thrown below. - auto self = move(_refToSelf4keepAlive); - - // Tell XrootD to release all resources associated with this request - Finished(); - - // Ask a subclass to process the response - // WARNING: This has to be the last call as the object may get deleted - // downstream. - proto::FrameBufferView view(_buf, _bufSize); - onResponse(view); - - } else { - // Double the buffer's capacity and copy over its previous content into the new location - int prevBufCapacity = _bufCapacity; - _bufIncrementSize = prevBufCapacity; - _bufCapacity += _bufIncrementSize; - - char* prevBuf = _buf; - _buf = new char[_bufCapacity]; - - copy(prevBuf, prevBuf + prevBufCapacity, _buf); - - delete[] prevBuf; - - // Keep reading - GetResponseData(_buf + _bufSize, _bufIncrementSize); - } - } -} - -} // namespace lsst::qserv::xrdreq diff --git a/src/xrdreq/QservRequest.h b/src/xrdreq/QservRequest.h deleted file mode 100644 index 4306d91311..0000000000 --- a/src/xrdreq/QservRequest.h +++ /dev/null @@ -1,120 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2011-2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDREQ_QSERV_REQUEST_H -#define LSST_QSERV_XRDREQ_QSERV_REQUEST_H - -// System headers -#include -#include -#include - -// Third party headers -#include "XrdSsi/XrdSsiRequest.hh" - -// Qserv headers -#include "proto/FrameBuffer.h" -#include "proto/worker.pb.h" - -namespace lsst::qserv::xrdreq { - -/** - * Class QservRequest is a base class for a family of the client-side requests - * (classes) to Qserv workers. - */ -class QservRequest : public XrdSsiRequest { -public: - QservRequest(QservRequest const&) = delete; - QservRequest& operator=(QservRequest const&) = delete; - virtual ~QservRequest() override; - - /** - * Do a proper request cancellation to ensure a pointer to the request gets deleted - * after calling XrdSsiRequest::Finished(true). - */ - void cancel(); - -protected: - QservRequest(); - - /** - * Setting a pointer to the object would guarantee that the life expectancy - * of the request be preserved before it's finished/failed and the corresponding - * notifications are sent to a subclass via the virtual methods QservRequest::onResponse() - * or QservRequest::onError(). The pointer will be reset after calling either of - * these methods, or the method QservRequest::cancel(). - * @param ptr The pointer to be set. - * @throws std::invalid_argument if the pointer is empty or pointing to a different - * request object. - */ - void setRefToSelf4keepAlive(std::shared_ptr ptr); - - /** - * Serialize a request into the provided buffer. The method is required to be - * provided by a subclass. - * @param buf A request buffer for serializing a request. - */ - virtual void onRequest(proto::FrameBuffer& buf) = 0; - - /** - * Process response from Qserv. The method is required to be provided by a subclass. - * @param view The buffer view for parsing results. - */ - virtual void onResponse(proto::FrameBufferView& view) = 0; - - /** - * Notify a base class about a failure occurred when sending a request data - * or receiving a response. - * @param error A message explaining a reason of the failure. - */ - virtual void onError(std::string const& msg) = 0; - - char* GetRequest(int& dlen) override; - bool ProcessResponse(const XrdSsiErrInfo& eInfo, const XrdSsiRespInfo& rInfo) override; - void ProcessResponseData(const XrdSsiErrInfo& eInfo, char* buff, int blen, bool last) override; - -private: - /// The global counter for the number of instances of any subclasses - static std::atomic _numClassInstances; - - /// Request buffer is prepared by subclasses before sending a request to a worker. - proto::FrameBuffer _frameBuf; - - // Response buffer is updated when receiving a response stream of data from a worker. - - /// The (very first and the) last increment of the capacity of the incoming - /// buffer is used to limit the amount of bytes to be received from a server. - int _bufIncrementSize; - - int _bufSize; ///< actual (meaningful) number of bytes in the incoming buffer - int _bufCapacity; ///< total capacity of the incoming buffer - - char* _buf; ///< buffer for incomming data - - /// The reference to the object is needed to guarantee the life expectency of - /// the request object while the request is still being processed. - std::shared_ptr _refToSelf4keepAlive; -}; - -} // namespace lsst::qserv::xrdreq - -#endif // LSST_QSERV_XRDREQ_QSERV_REQUEST_H \ No newline at end of file diff --git a/src/xrdreq/QueryManagementAction.cc b/src/xrdreq/QueryManagementAction.cc deleted file mode 100644 index f63a013b12..0000000000 --- a/src/xrdreq/QueryManagementAction.cc +++ /dev/null @@ -1,137 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "xrdreq/QueryManagementAction.h" - -// System headers -#include - -// Third party headers -#include "XrdCl/XrdClFile.hh" -#include "XrdCl/XrdClXRootDResponses.hh" -#include "XrdSsi/XrdSsiProvider.hh" -#include "XrdSsi/XrdSsiService.hh" - -// Qserv headers -#include "xrdreq/QueryManagementRequest.h" - -// LSST headers -#include "lsst/log/Log.h" - -/// This C++ symbol is provided by the SSI shared library -extern XrdSsiProvider* XrdSsiProviderClient; - -using namespace std; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdreq.QueryManagementAction"); - -string xrootdStatus2str(XrdCl::XRootDStatus const& s) { - return "status=" + to_string(s.status) + ", code=" + to_string(s.code) + ", errNo=" + to_string(s.errNo) + - ", message='" + s.GetErrorMessage() + "'"; -} - -/// The RAII wrapper around the silly C pointer to facilitate proper deletion -/// of the object returned by the XROOTD API. -struct LocationInfoRAII { - XrdCl::LocationInfo* locationInfo = nullptr; - ~LocationInfoRAII() { delete locationInfo; } -}; - -} // namespace - -namespace lsst::qserv::xrdreq { - -void QueryManagementAction::notifyAllWorkers(string const& xrootdFrontendUrl, - proto::QueryManagement::Operation op, uint32_t czarId, - QueryId queryId, CallbackType onFinish) { - auto const ptr = shared_ptr(new QueryManagementAction()); - ptr->_notifyAllWorkers(xrootdFrontendUrl, op, czarId, queryId, onFinish); -} - -QueryManagementAction::QueryManagementAction() { - LOGS(_log, LOG_LVL_TRACE, "QueryManagementAction ** CONSTRUCTED **"); -} - -QueryManagementAction::~QueryManagementAction() { - LOGS(_log, LOG_LVL_TRACE, "QueryManagementAction ** DELETED **"); -} - -void QueryManagementAction::_notifyAllWorkers(std::string const& xrootdFrontendUrl, - proto::QueryManagement::Operation op, uint32_t czarId, - QueryId queryId, CallbackType onFinish) { - string const context = "QueryManagementAction::" + string(__func__) + " "; - - // Find all subscribers (worker XROOTD servers) serving this special resource. - // Throw an exception if no workers are registered. - ::LocationInfoRAII locationInfoHandler; - string const queryResourceName = "/query"; - XrdCl::FileSystem fileSystem(xrootdFrontendUrl); - XrdCl::XRootDStatus const status = fileSystem.Locate(queryResourceName, XrdCl::OpenFlags::Flags::None, - locationInfoHandler.locationInfo); - if (!status.IsOK()) { - throw runtime_error(context + "failed to locate subscribers for resource " + queryResourceName + - ", " + ::xrootdStatus2str(status)); - } - if (uint32_t const numLocations = locationInfoHandler.locationInfo->GetSize(); numLocations == 0) { - throw runtime_error(context + "no subscribers are serving resource " + queryResourceName); - } else { - // Fill worker addresses as keys into the response object. - for (uint32_t i = 0; i < numLocations; ++i) { - _response[locationInfoHandler.locationInfo->At(i).GetAddress()] = string(); - } - } - - // Send a request to each worker. Note capturing a copy of 'self' to ensure - // the curent object will still existr while the requests will be being processed. - auto const self = shared_from_this(); - for (auto itr : _response) { - string const workerAddress = itr.first; - - // Connect to the worker service - XrdSsiErrInfo errInfo; - XrdSsiService* serviceProvider = XrdSsiProviderClient->GetService(errInfo, workerAddress); - if (nullptr == serviceProvider) { - throw runtime_error(context + " failed to contact worker service " + workerAddress + - ", error: " + errInfo.Get()); - } - - // Make and configure the request object - auto request = xrdreq::QueryManagementRequest::create( - op, czarId, queryId, - [self, workerAddress, onFinish](proto::WorkerCommandStatus::Code code, string const& error) { - if (code != proto::WorkerCommandStatus::SUCCESS) { - self->_response[workerAddress] = error; - } - if (++(self->_numWorkerRequestsFinished) == self->_response.size()) { - if (onFinish != nullptr) onFinish(self->_response); - } - }); - - // Initiate request processing - XrdSsiResource resource(queryResourceName); - serviceProvider->ProcessRequest(*request, resource); - } -} - -} // namespace lsst::qserv::xrdreq diff --git a/src/xrdreq/QueryManagementAction.h b/src/xrdreq/QueryManagementAction.h deleted file mode 100644 index f1779cae57..0000000000 --- a/src/xrdreq/QueryManagementAction.h +++ /dev/null @@ -1,96 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H -#define LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H - -// System headers -#include -#include -#include -#include -#include - -// Qserv headers -#include "global/intTypes.h" -#include "proto/worker.pb.h" - -namespace lsst::qserv::xrdreq { - -/** - * Class QueryManagementAction is an interface for managing query completion/cancellation - * at all Qserv workers that are connected as "publishers" to the XROOTD redirector. - */ -class QueryManagementAction : public std::enable_shared_from_this { -public: - /// The reponse type represents errors reported by the workers, where worker - /// names are the keys. And the values are the error messages. Empty strings - /// indicate the succesful completion of the requests. - using Response = std::map; - - /// The callback function type to be used for notifications on the operation completion. - using CallbackType = std::function; - - /** - * The front-end method for initiating the operation at all workers. - * - * @note The only way to track the completion of the requests sent via - * this interface is by providing the callback function. The request delivery - * is not guaranteeded in case if the XROOTD/SSI network will be clogged by - * the heavy traffic. It's safe to call the same operation many times if needed. - * - * @param xrootdFrontendUrl A location of the XROOTD redirector. - * @param op An operation be initiated at the workers. - * @param onFinish The optional callback to be fired upon the completion of - * the requested operation. - * - * @throws std::runtime_error For failures encountered when connecting to - * the manager or initiating the requesed operation. - */ - static void notifyAllWorkers(std::string const& xrootdFrontendUrl, proto::QueryManagement::Operation op, - uint32_t czarId, QueryId queryId, CallbackType onFinish = nullptr); - - QueryManagementAction(QueryManagementAction const&) = delete; - QueryManagementAction& operator=(QueryManagementAction const&) = delete; - virtual ~QueryManagementAction(); - -private: - QueryManagementAction(); - - /** - * The actual implementation of the request processor. - * @see QueryManagementAction::notifyAllWorkers() - */ - void _notifyAllWorkers(std::string const& xrootdFrontendUrl, proto::QueryManagement::Operation op, - uint32_t czarId, QueryId queryId, CallbackType onFinish); - - /// The collection of worker responses. - Response _response; - - /// The counter will get incremented as worker responses will be received. - /// User-provided callback function (if any) will be called when all requests - /// will finish (succeed or fail). - std::atomic _numWorkerRequestsFinished{0}; -}; - -} // namespace lsst::qserv::xrdreq - -#endif // LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H diff --git a/src/xrdreq/QueryManagementRequest.cc b/src/xrdreq/QueryManagementRequest.cc deleted file mode 100644 index 82860cdd59..0000000000 --- a/src/xrdreq/QueryManagementRequest.cc +++ /dev/null @@ -1,91 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "xrdreq/QueryManagementRequest.h" - -// LSST headers -#include "lsst/log/Log.h" - -using namespace std; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdreq.QueryManagementRequest"); -} // namespace - -namespace lsst::qserv::xrdreq { - -QueryManagementRequest::Ptr QueryManagementRequest::create(proto::QueryManagement::Operation op, - uint32_t czarId, QueryId queryId, - QueryManagementRequest::CallbackType onFinish) { - QueryManagementRequest::Ptr ptr(new QueryManagementRequest(op, czarId, queryId, onFinish)); - ptr->setRefToSelf4keepAlive(ptr); - return ptr; -} - -QueryManagementRequest::QueryManagementRequest(proto::QueryManagement::Operation op, uint32_t czarId, - QueryId queryId, QueryManagementRequest::CallbackType onFinish) - : _op(op), _czarId(czarId), _queryId(queryId), _onFinish(onFinish) { - LOGS(_log, LOG_LVL_TRACE, "QueryManagementRequest ** CONSTRUCTED **"); -} - -QueryManagementRequest::~QueryManagementRequest() { - LOGS(_log, LOG_LVL_TRACE, "QueryManagementRequest ** DELETED **"); -} - -void QueryManagementRequest::onRequest(proto::FrameBuffer& buf) { - proto::QueryManagement message; - message.set_op(_op); - message.set_czar_id(_czarId); - message.set_query_id(_queryId); - buf.serialize(message); -} - -void QueryManagementRequest::onResponse(proto::FrameBufferView& view) { - if (nullptr != _onFinish) { - // Clearing the stored callback after finishing the up-stream notification - // has two purposes: - // - // 1. it guaranties (exactly) one time notification - // 2. it breaks the up-stream dependency on a caller object if a shared - // pointer to the object was mentioned as the lambda-function's closure - auto onFinish = move(_onFinish); - _onFinish = nullptr; - onFinish(proto::WorkerCommandStatus::SUCCESS, string()); - } -} - -void QueryManagementRequest::onError(string const& error) { - if (nullptr != _onFinish) { - // Clearing the stored callback after finishing the up-stream notification - // has two purposes: - // - // 1. it guaranties (exactly) one time notification - // 2. it breaks the up-stream dependency on a caller object if a shared - // pointer to the object was mentioned as the lambda-function's closure - auto onFinish = move(_onFinish); - _onFinish = nullptr; - onFinish(proto::WorkerCommandStatus::ERROR, error); - } -} - -} // namespace lsst::qserv::xrdreq diff --git a/src/xrdreq/QueryManagementRequest.h b/src/xrdreq/QueryManagementRequest.h deleted file mode 100644 index 9c92fcfe6f..0000000000 --- a/src/xrdreq/QueryManagementRequest.h +++ /dev/null @@ -1,95 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_REQUEST_H -#define LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_REQUEST_H - -// System headers -#include -#include -#include - -// Qserv headers -#include "global/intTypes.h" -#include "proto/worker.pb.h" -#include "xrdreq/QservRequest.h" - -namespace lsst::qserv::xrdreq { - -/** - * Class QueryManagementRequest represents requests for managing query - * completion/cancellation at Qserv workers. - * @note No actuall responses are expected from these requests beyond - * the error messages in case of any problems in delivering or processing - * notifications. - */ -class QueryManagementRequest : public QservRequest { -public: - /// The pointer type for instances of the class - typedef std::shared_ptr Ptr; - - /// The callback function type to be used for notifications on - /// the operation completion. - using CallbackType = std::function; // error message (if failed) - - /** - * Static factory method is needed to prevent issues with the lifespan - * and memory management of instances created otherwise (as values or via - * low-level pointers). - * @param op An operation to be initiated. - * @param queryId An uinque identifier of a query affected by the request. - * Note that a cole of the identifier depends on which operation - * was requested. - * @param onFinish (optional) callback function to be called upon the completion - * (successful or not) of the request. - * @return the smart pointer to the object of the class - */ - static Ptr create(proto::QueryManagement::Operation op, uint32_t czarId, QueryId queryId, - CallbackType onFinish = nullptr); - - QueryManagementRequest() = delete; - QueryManagementRequest(QueryManagementRequest const&) = delete; - QueryManagementRequest& operator=(QueryManagementRequest const&) = delete; - - virtual ~QueryManagementRequest() override; - -protected: - /// @see QueryManagementRequest::create() - QueryManagementRequest(proto::QueryManagement::Operation op, uint32_t czarId, QueryId queryId, - CallbackType onFinish); - - virtual void onRequest(proto::FrameBuffer& buf) override; - virtual void onResponse(proto::FrameBufferView& view) override; - virtual void onError(std::string const& error) override; - -private: - // Parameters of the object - - proto::QueryManagement::Operation _op = proto::QueryManagement::CANCEL_AFTER_RESTART; - uint32_t _czarId = 0; - QueryId _queryId = 0; - CallbackType _onFinish; -}; - -} // namespace lsst::qserv::xrdreq - -#endif // LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_REQUEST_H diff --git a/src/xrdreq/qserv-query-management.cc b/src/xrdreq/qserv-query-management.cc deleted file mode 100644 index 0e410ff5ec..0000000000 --- a/src/xrdreq/qserv-query-management.cc +++ /dev/null @@ -1,154 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -// System header -#include -#include -#include -#include -#include - -// Third party headers -#include "XrdSsi/XrdSsiProvider.hh" -#include "XrdSsi/XrdSsiService.hh" - -// Qserv headers -#include "global/intTypes.h" -#include "proto/worker.pb.h" -#include "util/BlockPost.h" -#include "util/CmdLineParser.h" -#include "xrdreq/QueryManagementAction.h" -#include "xrdreq/QueryManagementRequest.h" - -/// This C++ symbol is provided by the SSI shared library -extern XrdSsiProvider* XrdSsiProviderClient; - -namespace global = lsst::qserv; -namespace proto = lsst::qserv::proto; -namespace util = lsst::qserv::util; -namespace xrdreq = lsst::qserv::xrdreq; - -using namespace std; - -namespace { - -// Command line parameters - -vector const allowedOperations = {"CANCEL_AFTER_RESTART", "CANCEL", "COMPLETE"}; -proto::QueryManagement::Operation operation = proto::QueryManagement::CANCEL_AFTER_RESTART; -uint32_t czarId; -global::QueryId queryId; -bool allWorkers = false; -string serviceProviderLocation; - -proto::QueryManagement::Operation str2operation(string const& str) { - if (str == "CANCEL_AFTER_RESTART") { - return proto::QueryManagement::CANCEL_AFTER_RESTART; - } else if (str == "CANCEL") { - return proto::QueryManagement::CANCEL; - } else if (str == "COMPLETE") { - return proto::QueryManagement::COMPLETE; - } - throw invalid_argument("error: unknown operation '" + str + "'"); -} - -int test() { - bool finished = false; - if (allWorkers) { - xrdreq::QueryManagementAction::notifyAllWorkers( - serviceProviderLocation, operation, czarId, queryId, - [&finished](xrdreq::QueryManagementAction::Response const& response) { - for (auto itr : response) { - cout << "worker: " << itr.first << " error: " << itr.second << endl; - } - finished = true; - }); - } else { - // Connect to a service provider - XrdSsiErrInfo errInfo; - auto serviceProvider = XrdSsiProviderClient->GetService(errInfo, serviceProviderLocation); - if (nullptr == serviceProvider) { - cerr << "failed to contact service provider at: " << serviceProviderLocation - << ", error: " << errInfo.Get() << endl; - return 1; - } - cout << "connected to service provider at: " << serviceProviderLocation << endl; - - // Prepare the request - auto request = xrdreq::QueryManagementRequest::create( - operation, czarId, queryId, - [&finished](proto::WorkerCommandStatus::Code code, string const& error) { - cout << "code=" << proto::WorkerCommandStatus_Code_Name(code) << ", error='" << error - << "'" << endl; - finished = true; - }); - - // Submit the request - XrdSsiResource resource("/query"); - serviceProvider->ProcessRequest(*request, resource); - } - - // Wait before the request will finish or fail - util::BlockPost blockPost(1000, 2000); - while (!finished) { - blockPost.wait(200); - } - return 0; -} -} // namespace - -int main(int argc, const char* const argv[]) { - // Verify that the version of the library that we linked against is - // compatible with the version of the headers we compiled against. - - GOOGLE_PROTOBUF_VERIFY_VERSION; - - // Parse command line parameters - try { - util::CmdLineParser parser( - argc, argv, - "\n" - "Usage:\n" - " \n" - " [--service=]\n" - "\n" - "Flags an options:\n" - " --all-workers - The flag indicating if the operation had to involve all workers.\n" - " --service= - A location of the service provider (default: 'localhost:1094').\n" - "\n" - "Parameters:\n" - " - An operation over the query (queries). Allowed values of\n" - " the parameter are: CANCEL_AFTER_RESTART, CANCEL, COMPLETE.\n" - " - The unique identifier of Czar.\n" - " - User query identifier.\n"); - - ::operation = ::str2operation(parser.parameterRestrictedBy(1, ::allowedOperations)); - ::czarId = parser.parameter(2); - ::queryId = parser.parameter(3); - ::allWorkers = parser.flag("all-workers"); - ::serviceProviderLocation = parser.option("service", "localhost:1094"); - - } catch (exception const& ex) { - cerr << ex.what() << endl; - return 1; - } - return ::test(); -} diff --git a/src/xrdsvc/CMakeLists.txt b/src/xrdsvc/CMakeLists.txt index 2e5455bab1..818455ad75 100644 --- a/src/xrdsvc/CMakeLists.txt +++ b/src/xrdsvc/CMakeLists.txt @@ -1,17 +1,14 @@ add_library(xrdsvc SHARED) add_dependencies(xrdsvc proto) -target_sources(xrdsvc PRIVATE - ChannelStream.cc +target_sources(qserv_xrdsvc PRIVATE HttpModule.cc HttpMonitorModule.cc HttpReplicaMgtModule.cc HttpWorkerCzarModule.cc HttpSvc.cc SsiProvider.cc - SsiRequest.cc SsiService.cc - StreamBuffer.cc ) target_include_directories(xrdsvc PRIVATE diff --git a/src/xrdsvc/ChannelStream.cc b/src/xrdsvc/ChannelStream.cc deleted file mode 100644 index 2c02610b48..0000000000 --- a/src/xrdsvc/ChannelStream.cc +++ /dev/null @@ -1,115 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "xrdsvc/ChannelStream.h" - -// Third-party headers -#include "boost/utility.hpp" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "global/debugUtil.h" -#include "util/Bug.h" -#include "util/common.h" - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.ChannelStream"); -} - -using namespace std; - -namespace lsst::qserv::xrdsvc { - -/// Provide each Channel stream with a unique identifier. -atomic ChannelStream::_sequenceSource{0}; - -/// Constructor -ChannelStream::ChannelStream() : XrdSsiStream(isActive), _closed(false), _seq(_sequenceSource++) {} - -/// Destructor -ChannelStream::~ChannelStream() { clearMsgs(); } - -/// Push in a data packet -void ChannelStream::append(StreamBuffer::Ptr const &streamBuffer, bool last) { - if (_closed) { - throw util::Bug(ERR_LOC, - "ChannelStream::append: Stream closed, append(...,last=true) already received"); - } - LOGS(_log, LOG_LVL_DEBUG, - "seq=" << _seq << " ChannelStream::append last=" << last << " " - << util::prettyCharBuf(streamBuffer->data, streamBuffer->getSize(), 5)); - { - unique_lock lock(_mutex); - ++_appendCount; - LOGS(_log, LOG_LVL_DEBUG, - "seq=" << to_string(_seq) << " Trying to append message (flowing) appC=" << _appendCount - << " getBC=" << _getBufCount); - _msgs.push_back(streamBuffer); - _closed = last; // if last is true, then we are closed. - } - _hasDataCondition.notify_one(); -} - -/// Pull out a data packet as a Buffer object (called by XrdSsi code) -XrdSsiStream::Buffer *ChannelStream::GetBuff(XrdSsiErrInfo &eInfo, int &dlen, bool &last) { - ++_getBufCount; - // This InstanceCount should be fairly quiet as there should only be one at a time. - util::InstanceCount inst("GetBuf seq=" + to_string(_seq)); - unique_lock lock(_mutex); - while (_msgs.empty() && !_closed) { // No msgs, but we aren't done - // wait. - LOGS(_log, LOG_LVL_INFO, "seq=" << _seq << " Waiting, no data ready "); - _hasDataCondition.wait(lock); - } - if (_msgs.empty() && _closed) { - // It's closed and no more msgs are available. - LOGS(_log, LOG_LVL_INFO, "seq=" << _seq << " Not waiting, but closed"); - dlen = 0; - eInfo.Set("Not an active stream", EOPNOTSUPP); - return 0; - } - - StreamBuffer::Ptr sb = _msgs.front(); - dlen = sb->getSize(); - _msgs.pop_front(); - last = _closed && _msgs.empty(); - LOGS(_log, LOG_LVL_INFO, - "seq=" << to_string(_seq) << " returning buffer (" << dlen << ", " << (last ? "(last)" : "(more)") - << ")" - << " getBufCount=" << _getBufCount); - return sb.get(); -} - -void ChannelStream::clearMsgs() { - LOGS(_log, LOG_LVL_DEBUG, "seq=" << to_string(_seq) << " ChannelStream::clearMsgs()"); - unique_lock lock(_mutex); - while (!_msgs.empty()) { - _msgs.front()->Recycle(); - _msgs.pop_front(); - } -} - -} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/ChannelStream.h b/src/xrdsvc/ChannelStream.h deleted file mode 100644 index ee2de6005d..0000000000 --- a/src/xrdsvc/ChannelStream.h +++ /dev/null @@ -1,75 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDSVC_CHANNELSTREAM_H -#define LSST_QSERV_XRDSVC_CHANNELSTREAM_H - -// System headers -#include -#include -#include -#include - -// qserv headers -#include "xrdsvc/StreamBuffer.h" - -// Third-party headers -#include "XrdSsi/XrdSsiErrInfo.hh" // required by XrdSsiStream -#include "XrdSsi/XrdSsiStream.hh" - -namespace lsst::qserv::xrdsvc { - -/// ChannelStream is an implementation of an XrdSsiStream that accepts -/// SendChannel streamed data. -class ChannelStream : public XrdSsiStream { -public: - ChannelStream(); - virtual ~ChannelStream(); - - /// Push in a data packet - void append(StreamBuffer::Ptr const &StreamBuffer, bool last); - - /// Empty _msgs, calling StreamBuffer::Recycle() where needed. - void clearMsgs(); - - /// Pull out a data packet as a Buffer object (called by XrdSsi code) - Buffer *GetBuff(XrdSsiErrInfo &eInfo, int &dlen, bool &last) override; - - bool closed() const { return _closed; } - - uint64_t getSeq() const { return _seq; } - -private: - bool _closed; ///< Closed to new append() calls? - // Can keep a deque of (buf, bufsize) to reduce copying, if needed. - std::deque _msgs; ///< Message queue - std::mutex _mutex; ///< _msgs protection - std::condition_variable _hasDataCondition; ///< _msgs condition - uint64_t const _seq; ///< Unique identifier for this instance. - static std::atomic _sequenceSource; ///< Source of unique identifiers. - std::atomic _appendCount{0}; ///< number of appends - std::atomic _getBufCount{0}; ///< number of buffers -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDSVC_CHANNELSTREAM_H diff --git a/src/xrdsvc/HttpReplicaMgtModule.cc b/src/xrdsvc/HttpReplicaMgtModule.cc index 14fdde32af..91692aa921 100644 --- a/src/xrdsvc/HttpReplicaMgtModule.cc +++ b/src/xrdsvc/HttpReplicaMgtModule.cc @@ -78,10 +78,7 @@ HttpReplicaMgtModule::HttpReplicaMgtModule(string const& context, shared_ptr const& foreman, shared_ptr const& req, shared_ptr const& resp) - : HttpModule(context, foreman, req, resp), - _providerServer(dynamic_cast(XrdSsiProviderLookup)), - _clusterManager(_providerServer->GetClusterManager()), - _dataContext(_clusterManager->DataContext()) {} + : HttpModule(context, foreman, req, resp) {} json HttpReplicaMgtModule::executeImpl(string const& subModuleName) { string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; @@ -334,12 +331,8 @@ void HttpReplicaMgtModule::_modifyChunk(string const& func, int chunk, string co // copy of the inventory. After that modify both (persistent and // transient) inventories. if (Direction::ADD == direction) { - _clusterManager->Added(resource.data()); - if (_dataContext) _providerServer->GetChunkInventory().add(database, chunk); foreman()->chunkInventory()->add(database, chunk, foreman()->mySqlConfig()); } else { - _clusterManager->Removed(resource.data()); - if (_dataContext) _providerServer->GetChunkInventory().remove(database, chunk); foreman()->chunkInventory()->remove(database, chunk, foreman()->mySqlConfig()); } } catch (wpublish::InvalidParamError const& ex) { diff --git a/src/xrdsvc/HttpReplicaMgtModule.h b/src/xrdsvc/HttpReplicaMgtModule.h index efda8acfff..ac58a58283 100644 --- a/src/xrdsvc/HttpReplicaMgtModule.h +++ b/src/xrdsvc/HttpReplicaMgtModule.h @@ -184,10 +184,6 @@ class HttpReplicaMgtModule : public xrdsvc::HttpModule { */ void _modifyChunk(std::string const& func, int chunk, std::string const& database, Direction direction); - // XROOTD/SSI service context. - - xrdsvc::SsiProviderServer* _providerServer = nullptr; - XrdSsiCluster* _clusterManager = nullptr; bool _dataContext = false; }; diff --git a/src/xrdsvc/HttpSvc.cc b/src/xrdsvc/HttpSvc.cc index 49781fc24e..0908efcaaa 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/xrdsvc/HttpSvc.cc @@ -138,7 +138,13 @@ uint16_t HttpSvc::start() { _httpServerPtr->addHandlers( {{"POST", "/queryjob", [self](shared_ptr const& req, shared_ptr const& resp) { - HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "QUERYJOB", + HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "/queryjob", + http::AuthType::REQUIRED); + }}}); + _httpServerPtr->addHandlers( + {{"POST", "/querystatus", + [self](shared_ptr const& req, shared_ptr const& resp) { + HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "/querystatus", http::AuthType::REQUIRED); }}}); _httpServerPtr->start(); diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 8c0748b5ae..8a4aa910b1 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -34,9 +34,11 @@ #include "http/Client.h" // TODO:UJ will probably need to be removed #include "http/Exceptions.h" #include "http/MetaModule.h" -#include "http/RequestBody.h" +#include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" #include "mysql/MySqlUtils.h" +#include "protojson/UberJobMsg.h" +#include "protojson/WorkerQueryStatusData.h" #include "qmeta/types.h" #include "util/String.h" #include "util/Timer.h" @@ -44,10 +46,13 @@ #include "wbase/Task.h" #include "wbase/UberJobData.h" #include "wbase/UserQueryInfo.h" -#include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" +#include "wcontrol/WCzarInfoMap.h" +#include "wconfig/WorkerConfig.h" #include "wcontrol/ResourceMonitor.h" #include "wpublish/ChunkInventory.h" +#include "wpublish/QueriesAndChunks.h" +#include "wpublish/QueryStatistics.h" #include "xrdsvc/SsiProvider.h" #include "xrdsvc/XrdName.h" @@ -88,7 +93,8 @@ json HttpWorkerCzarModule::executeImpl(string const& subModuleName) { string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; enforceInstanceId(func, wconfig::WorkerConfig::instance()->replicationInstanceId()); enforceWorkerId(func); - if (subModuleName == "QUERYJOB") return _queryJob(); + if (subModuleName == "/queryjob") return _queryJob(); + if (subModuleName == "/querystatus") return _queryStatus(); throw invalid_argument(context() + func + " unsupported sub-module"); } @@ -104,94 +110,52 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { json jsRet; vector ujTasks; try { - // See qdisp::UberJob::runUberJob() for json message construction. auto const& jsReq = body().objJson; - string const targetWorkerId = body().required("worker"); - - http::RequestBody rbCzar(body().required("czar")); - auto czarName = rbCzar.required("name"); - auto czarId = rbCzar.required("id"); - auto czarPort = rbCzar.required("management-port"); - auto czarHostName = rbCzar.required("management-host-name"); - LOGS(_log, LOG_LVL_TRACE, - __func__ << " czar n=" << czarName << " id=" << czarId << " p=" << czarPort - << " h=" << czarHostName); - - http::RequestBody rbUberJob(body().required("uberjob")); - auto ujQueryId = rbUberJob.required("queryid"); - auto ujId = rbUberJob.required("uberjobid"); - auto ujCzarId = rbUberJob.required("czarid"); - auto ujJobs = rbUberJob.required("jobs"); - LOGS(_log, LOG_LVL_TRACE, - __func__ << " uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); - - auto ujData = wbase::UberJobData::create(ujId, czarName, czarId, czarHostName, czarPort, ujQueryId, - targetWorkerId, foreman(), authKey()); - - // Find the entry for this queryId, creat a new one if needed. - wbase::UserQueryInfo::Ptr userQueryInfo = wbase::UserQueryInfo::uqMapInsert(ujQueryId); - userQueryInfo->addUberJob(ujData); + auto uberJobMsg = protojson::UberJobMsg::createFromJson(jsReq); - auto channelShared = - wbase::FileChannelShared::create(ujData, czarId, czarHostName, czarPort, targetWorkerId); - ujData->setFileChannelShared(channelShared); + UberJobId ujId = uberJobMsg->getUberJobId(); + auto ujCzInfo = uberJobMsg->getCzarContactInfo(); + auto czarId = ujCzInfo->czId; + QueryId ujQueryId = uberJobMsg->getQueryId(); + int ujRowLimit = uberJobMsg->getRowLimit(); + auto targetWorkerId = uberJobMsg->getWorkerId(); - // TODO:UJ These items should be stored higher in the message structure as they get - // duplicated and should always be the same within an UberJob. - QueryId jdQueryId = 0; - proto::ScanInfo scanInfo; - bool scanInfoSet = false; - bool jdScanInteractive = false; - int jdMaxTableSize = 0; - - for (auto const& job : ujJobs) { - json const& jsJobDesc = job["jobdesc"]; - http::RequestBody rbJobDesc(jsJobDesc); - // See qproc::TaskMsgFactory::makeMsgJson for message construction. - auto const jdCzarId = rbJobDesc.required("czarId"); - jdQueryId = rbJobDesc.required("queryId"); - auto const jdJobId = rbJobDesc.required("jobId"); - auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - auto const jdScanPriority = rbJobDesc.required("scanPriority"); - jdScanInteractive = rbJobDesc.required("scanInteractive"); - jdMaxTableSize = rbJobDesc.required("maxTableSize"); - auto const jdChunkId = rbJobDesc.required("chunkId"); - LOGS(_log, LOG_LVL_TRACE, - __func__ << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId - << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive - << " maxTblSz=" << jdMaxTableSize << " chunkId=" << jdChunkId); - - auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); - if (!scanInfoSet) { - for (auto const& tbl : jdChunkScanTables) { - http::RequestBody rbTbl(tbl); - auto const& chunkScanDb = rbTbl.required("db"); - auto const& lockInMemory = rbTbl.required("lockInMemory"); - auto const& chunkScanTable = rbTbl.required("table"); - auto const& tblScanRating = rbTbl.required("tblScanRating"); - LOGS(_log, LOG_LVL_TRACE, - __func__ << " chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory - << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); - scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, - tblScanRating); - scanInfoSet = true; - } - } - scanInfo.scanRating = jdScanPriority; + // Get or create QueryStatistics and UserQueryInfo instances. + auto queryStats = foreman()->getQueriesAndChunks()->addQueryId(ujQueryId, ujCzInfo->czId); + auto userQueryInfo = queryStats->getUserQueryInfo(); + + if (userQueryInfo->getCancelledByCzar()) { + throw wbase::TaskException( + ERR_LOC, string("Already cancelled by czar. ujQueryId=") + to_string(ujQueryId)); + } + if (userQueryInfo->isUberJobDead(ujId)) { + throw wbase::TaskException(ERR_LOC, string("UberJob already dead. ujQueryId=") + + to_string(ujQueryId) + " ujId=" + to_string(ujId)); } - // create tasks and add them to ujData - auto chunkTasks = wbase::Task::createTasksForChunk( - ujData, ujJobs, channelShared, scanInfo, jdScanInteractive, jdMaxTableSize, - foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), foreman()->sqlConnMgr(), - foreman()->queriesAndChunks(), foreman()->httpPort()); - ujTasks.insert(ujTasks.end(), chunkTasks.begin(), chunkTasks.end()); + auto ujData = wbase::UberJobData::create(ujId, ujCzInfo->czName, ujCzInfo->czId, ujCzInfo->czHostName, + ujCzInfo->czPort, ujQueryId, ujRowLimit, targetWorkerId, + foreman(), authKey()); + + // Find the entry for this queryId, create a new one if needed. + userQueryInfo->addUberJob(ujData); + auto channelShared = wbase::FileChannelShared::create(ujData, ujCzInfo->czId, ujCzInfo->czHostName, + ujCzInfo->czPort, targetWorkerId); + + ujData->setFileChannelShared(channelShared); + auto ujTasks = wbase::Task::createTasksFromUberJobMsg( + uberJobMsg, ujData, channelShared, foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), + foreman()->sqlConnMgr(), foreman()->queriesAndChunks(), foreman()->httpPort()); channelShared->setTaskCount(ujTasks.size()); ujData->addTasks(ujTasks); + // At this point, it looks like the message was sent successfully, update + // czar touched time. + wcontrol::WCzarInfoMap::Ptr wCzarMap = foreman()->getWCzarInfoMap(); + wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czarId); + wCzarInfo->czarMsgReceived(CLOCK::now()); + util::Timer timer; timer.start(); foreman()->processTasks(ujTasks); // Queues tasks to be run later. @@ -210,4 +174,115 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { return jsRet; } +json HttpWorkerCzarModule::_queryStatus() { + debug(__func__); + checkApiVersion(__func__, 34); + // At this point, API version, correct worker, and auth have been checked. + json jsRet = _handleQueryStatus(__func__); + return jsRet; +} + +json HttpWorkerCzarModule::_handleQueryStatus(std::string const& func) { + json jsRet; + auto now = CLOCK::now(); + auto const workerConfig = wconfig::WorkerConfig::instance(); + auto const replicationInstanceId = workerConfig->replicationInstanceId(); + auto const replicationAuthKey = workerConfig->replicationAuthKey(); + + auto const& jsReq = body().objJson; + auto wqsData = protojson::WorkerQueryStatusData::createFromJson(jsReq, replicationInstanceId, + replicationAuthKey, now); + + auto const czInfo = wqsData->getCzInfo(); + LOGS(_log, LOG_LVL_TRACE, " HttpWorkerCzarModule::_handleQueryStatus req=" << jsReq.dump()); + CzarIdType czId = czInfo->czId; + wcontrol::WCzarInfoMap::Ptr wCzarMap = foreman()->getWCzarInfoMap(); + wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czId); + wCzarInfo->czarMsgReceived(CLOCK::now()); + + // For all queryId and czarId items, if the item can't be found, it is simply ignored. Anything that + // is missed will eventually be picked up by other mechanisms, such as results being rejected + // by the czar. This almost never happen, but the system should respond gracefully. + + // If a czar was restarted, cancel and delete the abandoned items. + if (wqsData->isCzarRestart()) { + auto restartCzarId = wqsData->getCzarRestartCzarId(); + auto restartQId = wqsData->getCzarRestartQueryId(); + if (restartCzarId > 0 && restartQId > 0) { + wbase::FileChannelShared::cleanUpResultsOnCzarRestart(wqsData->getCzarRestartCzarId(), + wqsData->getCzarRestartQueryId()); + } + } + + // Take the values from the lists in the message to cancel the + // appropriate queries and tasks as needed. + auto const queriesAndChunks = foreman()->queriesAndChunks(); + vector cancelledList; + vector deleteFilesList; + { + // Cancelled queries where we want to keep the files + lock_guard mapLg(wqsData->mapMtx); + for (auto const& [dkQid, dkTm] : wqsData->qIdDoneKeepFiles) { + auto qStats = queriesAndChunks->addQueryId(dkQid, czId); + if (qStats != nullptr) { + auto uqInfo = qStats->getUserQueryInfo(); + if (uqInfo != nullptr) { + if (!uqInfo->getCancelledByCzar()) { + cancelledList.push_back(uqInfo); + } + } + } + } + for (auto const& [dkQid, dkTm] : wqsData->qIdDoneDeleteFiles) { + auto qStats = queriesAndChunks->addQueryId(dkQid, czId); + if (qStats != nullptr) { + auto uqInfo = qStats->getUserQueryInfo(); + if (uqInfo != nullptr) { + if (!uqInfo->getCancelledByCzar()) { + cancelledList.push_back(uqInfo); + } + deleteFilesList.push_back(uqInfo); + } + } + } + } + + // Cancel everything in the cancelled list. + for (auto const& canUqInfo : cancelledList) { + canUqInfo->cancelFromCzar(); + } + + // For dead UberJobs, add them to a list of dead uberjobs within UserQueryInfo. + // UserQueryInfo will cancel the tasks in the uberjobs if they exist. + // New UberJob Id's will be checked against the list, and immediately be + // killed if they are on it. (see HttpWorkerCzarModule::_handleQueryJob) + for (auto const& [ujQid, ujIdMap] : wqsData->qIdDeadUberJobs) { + auto qStats = queriesAndChunks->addQueryId(ujQid, czId); + if (qStats != nullptr) { + auto uqInfo = qStats->getUserQueryInfo(); + if (uqInfo != nullptr) { + if (!uqInfo->getCancelledByCzar()) { + for (auto const& [ujId, tm] : ujIdMap) { + uqInfo->cancelUberJob(ujId); + } + } + } + } + } + + // Delete files that should be deleted + CzarIdType czarId = wqsData->getCzInfo()->czId; + for (wbase::UserQueryInfo::Ptr uqiPtr : deleteFilesList) { + if (uqiPtr == nullptr) continue; + QueryId qId = uqiPtr->getQueryId(); + wbase::FileChannelShared::cleanUpResults(czarId, qId); + } + // Syntax errors in the message would throw invalid_argument, which is handled elsewhere. + + // Return a message containing lists of the queries that were cancelled. + jsRet = wqsData->serializeResponseJson(foreman()->getWorkerStartupTime()); + wCzarInfo->sendWorkerCzarComIssueIfNeeded(wqsData->getWInfo(), wqsData->getCzInfo()); + return jsRet; +} + } // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/xrdsvc/HttpWorkerCzarModule.h index bb75a63c55..500c905e5c 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.h +++ b/src/xrdsvc/HttpWorkerCzarModule.h @@ -81,6 +81,14 @@ class HttpWorkerCzarModule : public xrdsvc::HttpModule { /// Handle an UberJob message from the czar to run it on this worker, this does /// work of deciphering the message, creating UberJobData objects and Task objects. nlohmann::json _handleQueryJob(std::string const& func); + + /// Verify some aspects of the query and call _handleQueryStatus + nlohmann::json _queryStatus(); + + /// Reconstruct the message, absorb the lists into this worker's state, + /// queue the ComIssue message and needed, and send the lists back to + /// the czar. + nlohmann::json _handleQueryStatus(std::string const& func); }; } // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/SsiProvider.cc b/src/xrdsvc/SsiProvider.cc index f7a0684110..53463b29e2 100644 --- a/src/xrdsvc/SsiProvider.cc +++ b/src/xrdsvc/SsiProvider.cc @@ -146,64 +146,17 @@ bool SsiProviderServer::Init(XrdSsiLogger* logP, XrdSsiCluster* clsP, std::strin XrdSsiProvider::rStat SsiProviderServer::QueryResource(char const* rName, char const* contact) { // Validate resource name based on its proposed type - - ResourceUnit ru(rName); - if (ru.unitType() == ResourceUnit::DBCHUNK) { - // Extract db and chunk from path and validate result - - // If the chunk exists on our node then tell the caller it is here. - if (_chunkInventory.has(ru.db(), ru.chunk())) { - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider Query " << rName << " present"); - return isPresent; - } - - // Tell the caller we do not have the chunk. - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider Query " << rName << " absent"); - return notPresent; - } else if (ru.unitType() == ResourceUnit::QUERY) { - return isPresent; - } - - // Treat other resources as absolute path names of files - boost::filesystem::path const path(rName); - if (path.is_absolute()) { - boost::system::error_code ec; - if (boost::filesystem::exists(path, ec) && !ec.value()) { - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider File Resource " << rName << " recognized"); - return isPresent; - } - } - - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider Query " << rName << " invalid"); return notPresent; } void SsiProviderServer::ResourceAdded(const char* rName) { // Handle resource based on its proposed type - - ResourceUnit ru(rName); - if (ru.unitType() == ResourceUnit::DBCHUNK) { - // Extract db and chunk from path and add the resource to the chunk - // inventory - _chunkInventory.add(ru.db(), ru.chunk()); - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider ResourceAdded " << rName); - return; - } - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider ResourceAdded " << rName << " invalid"); + return; } void SsiProviderServer::ResourceRemoved(const char* rName) { // Handle resource based on its proposed type - - ResourceUnit ru(rName); - if (ru.unitType() == ResourceUnit::DBCHUNK) { - // Extract db and chunk from path and add the resource to the chunk - // inventory - _chunkInventory.remove(ru.db(), ru.chunk()); - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider ResourceRemoved " << rName); - return; - } - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider ResourceRemoved " << rName << " invalid"); + return; } } // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/SsiRequest.cc b/src/xrdsvc/SsiRequest.cc deleted file mode 100644 index c1c9da2341..0000000000 --- a/src/xrdsvc/SsiRequest.cc +++ /dev/null @@ -1,389 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include -#include -#include -#include -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiRequest.hh" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "global/intTypes.h" -#include "global/LogContext.h" -#include "global/ResourceUnit.h" -#include "proto/FrameBuffer.h" -#include "proto/worker.pb.h" -#include "util/InstanceCount.h" -#include "util/HoldTrack.h" -#include "util/Timer.h" -#include "wbase/FileChannelShared.h" -#include "wbase/TaskState.h" -#include "wbase/Task.h" -#include "wconfig/WorkerConfig.h" -#include "wcontrol/Foreman.h" -#include "wcontrol/ResourceMonitor.h" -#include "wpublish/ChunkInventory.h" -#include "xrdsvc/ChannelStream.h" - -namespace proto = lsst::qserv::proto; -namespace wbase = lsst::qserv::wbase; - -namespace { - -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiRequest"); - -} // namespace - -namespace lsst::qserv::xrdsvc { - -SsiRequest::Ptr SsiRequest::newSsiRequest(std::string const& rname, - std::shared_ptr const& foreman) { - auto req = SsiRequest::Ptr(new SsiRequest(rname, foreman)); - req->_selfKeepAlive = req; - return req; -} - -SsiRequest::SsiRequest(std::string const& rname, std::shared_ptr const& foreman) - : _validator(foreman->chunkInventory()->newValidator()), _foreman(foreman), _resourceName(rname) {} - -SsiRequest::~SsiRequest() { - LOGS(_log, LOG_LVL_DEBUG, "~SsiRequest()"); - UnBindRequest(); -} - -void SsiRequest::reportError(std::string const& errStr) { - LOGS(_log, LOG_LVL_WARN, errStr); - replyError(errStr, EINVAL); - ReleaseRequestBuffer(); -} - -uint64_t countLimiter = 0; // LockupDB - -// Step 4 -/// Called by XrdSsi to actually process a request. -void SsiRequest::execute(XrdSsiRequest& req) { - util::Timer t; - LOGS(_log, LOG_LVL_DEBUG, "Execute request, resource=" << _resourceName); - - char* reqData = nullptr; - int reqSize; - t.start(); - reqData = req.GetRequest(reqSize); - t.stop(); - LOGS(_log, LOG_LVL_DEBUG, "GetRequest took " << t.getElapsed() << " seconds"); - - // We bind this object to the request now. This allows us to respond at any - // time (much simpler). Though the manual forgot to say that all pending - // events will be reflected on a different thread the moment we bind the - // request; the fact allows us to use a mutex to serialize the order of - // initialization and possible early cancellation. We protect this code - // with a mutex gaurd which will be released upon exit. - // - std::lock_guard lock(_finMutex); - BindRequest(req); - - ResourceUnit ru(_resourceName); - - // Make sure the requested resource belongs to this worker - if (!(*_validator)(ru)) { - reportError("WARNING: request to the unowned resource detected:" + _resourceName); - return; - } - - auto const sendChannel = std::make_shared(shared_from_this()); - - // Process the request - switch (ru.unitType()) { - case ResourceUnit::DBCHUNK: { - // Increment the counter of the database/chunk resources in use - _foreman->resourceMonitor()->increment(_resourceName); - - // reqData has the entire request, so we can unpack it without waiting for - // more data. - LOGS(_log, LOG_LVL_DEBUG, "Decoding TaskMsg of size " << reqSize); - auto taskMsg = std::make_shared(); - if (!taskMsg->ParseFromArray(reqData, reqSize) || !taskMsg->IsInitialized()) { - reportError("Failed to decode TaskMsg on resource db=" + ru.db() + - " chunkId=" + std::to_string(ru.chunk())); - return; - } - - QSERV_LOGCONTEXT_QUERY_JOB(taskMsg->queryid(), taskMsg->jobid()); - - if (!taskMsg->has_db() || !taskMsg->has_chunkid() || (ru.db() != taskMsg->db()) || - (ru.chunk() != taskMsg->chunkid())) { - reportError("Mismatched db/chunk in TaskMsg on resource db=" + ru.db() + - " chunkId=" + std::to_string(ru.chunk())); - return; - } - - if (not(taskMsg->has_queryid() && taskMsg->has_jobid() && taskMsg->has_scaninteractive() && - taskMsg->has_attemptcount() && taskMsg->has_czarid())) { - reportError(std::string("taskMsg missing required field ") + - " queryid:" + std::to_string(taskMsg->has_queryid()) + - " jobid:" + std::to_string(taskMsg->has_jobid()) + - " scaninteractive:" + std::to_string(taskMsg->has_scaninteractive()) + - " attemptcount:" + std::to_string(taskMsg->has_attemptcount()) + - " czarid:" + std::to_string(taskMsg->has_czarid())); - return; - } - _channelShared = wbase::FileChannelShared::create(sendChannel, taskMsg->czarid(), - _foreman->chunkInventory()->id()); - auto const tasks = wbase::Task::createTasks(taskMsg, _channelShared, _foreman->chunkResourceMgr(), - _foreman->mySqlConfig(), _foreman->sqlConnMgr(), - _foreman->queriesAndChunks(), _foreman->httpPort()); - for (auto const& task : tasks) { - _tasks.push_back(task); - } - - // Now that the request is decoded (successfully or not), release the - // xrootd request buffer. To avoid data races, this must happen before - // the task is handed off to another thread for processing, as there is a - // reference to this SsiRequest inside the reply channel for the task, - // and after the call to BindRequest. - ReleaseRequestBuffer(); - t.start(); - _foreman->processTasks(tasks); // Queues tasks to be run later. - t.stop(); - LOGS(_log, LOG_LVL_DEBUG, - "Enqueued TaskMsg for " << ru << " in " << t.getElapsed() << " seconds"); - break; - } - case ResourceUnit::QUERY: { - LOGS(_log, LOG_LVL_DEBUG, "Parsing request details for resource=" << _resourceName); - proto::QueryManagement request; - try { - // reqData has the entire request, so we can unpack it without waiting for - // more data. - proto::FrameBufferView view(reqData, reqSize); - view.parse(request); - ReleaseRequestBuffer(); - } catch (proto::FrameBufferError const& ex) { - reportError("Failed to decode a query completion/cancellation command, error: " + - std::string(ex.what())); - break; - } - LOGS(_log, LOG_LVL_DEBUG, - "QueryManagement: op=" << proto::QueryManagement_Operation_Name(request.op()) - << " query_id=" << request.query_id()); - - switch (request.op()) { - case proto::QueryManagement::CANCEL_AFTER_RESTART: - // TODO: locate and cancel the coresponding tasks, remove the tasks - // from the scheduler queues. - wbase::FileChannelShared::cleanUpResultsOnCzarRestart(request.czar_id(), - request.query_id()); - break; - case proto::QueryManagement::CANCEL: - // TODO: locate and cancel the coresponding tasks, remove the tasks - // from the scheduler queues. - wbase::FileChannelShared::cleanUpResults(request.czar_id(), request.query_id()); - break; - case proto::QueryManagement::COMPLETE: - wbase::FileChannelShared::cleanUpResults(request.czar_id(), request.query_id()); - break; - default: - reportError("QueryManagement: op=" + proto::QueryManagement_Operation_Name(request.op()) + - " is not supported by the current implementation."); - return; - } - - // Send back the empty response since no info is expected by a caller - // for this type of requests beyond the usual error notifications (if any). - this->reply((char const*)0, 0); - break; - } - default: - reportError("Unexpected unit type '" + std::to_string(ru.unitType()) + - "', resource name: " + _resourceName); - break; - } - - // Note that upon exit the _finMutex will be unlocked allowing Finished() - // to actually do something once everything is actually setup. -} - -/// Called by SSI to free resources. -void SsiRequest::Finished(XrdSsiRequest& req, XrdSsiRespInfo const& rinfo, bool cancel) { // Step 8 - util::HoldTrack::Mark markA(ERR_LOC, "SsiRequest::Finished start"); - if (cancel) { - // Either the czar of xrootd has decided to cancel the Job. - // Try to cancel all of the tasks, if there are any. - for (auto&& wTask : _tasks) { - auto task = wTask.lock(); - if (task != nullptr) { - task->cancel(); - } - } - } - - // This call is sync (blocking). - // client finished retrieving response, or cancelled. - // release response resources (e.g. buf) - // But first we must make sure that request setup completed (i.e execute()) by - // locking _finMutex. - { - std::lock_guard finLock(_finMutex); - // Clean up _stream if it exists and don't add anything new to it either. - _reqFinished = true; - if (_stream != nullptr) { - _stream->clearMsgs(); - } - } - - // This will clear the cyclic dependency: - // FileChannelShared -> ChannelStream -> SsiRequest -> FileChannelShared - // - // TODO: Eliminate xrdsvc::ChannelStream sinve this class seems to be useless - // in the file-based result delivery protocol. - _channelShared.reset(); - - auto keepAlive = freeSelfKeepAlive(); - - // No buffers allocated, so don't need to free. - // We can release/unlink the file now - const char* type = ""; - switch (rinfo.rType) { - case XrdSsiRespInfo::isNone: - type = "type=isNone"; - break; - case XrdSsiRespInfo::isData: - type = "type=isData"; - break; - case XrdSsiRespInfo::isError: - type = "type=isError"; - break; - case XrdSsiRespInfo::isFile: - type = "type=isFile"; - break; - case XrdSsiRespInfo::isStream: - type = "type=isStream"; - break; - case XrdSsiRespInfo::isHandle: - type = "type=isHandle"; - break; - } - - // Decrement the counter of the database/chunk resources in use - ResourceUnit ru(_resourceName); - if (ru.unitType() == ResourceUnit::DBCHUNK) { - _foreman->resourceMonitor()->decrement(_resourceName); - } - - // We can't do much other than close the file. - // It should work (on linux) to unlink the file after we open it, though. - // With the optimizer on '-Og', there was a double free for a SsiRequest. - // The likely cause could be keepAlive being optimized out for being unused. - // The problem has not reoccurred since adding keepAlive to the following - // comment, but having code depend on a comment line is ugly in its own way. - LOGS(_log, LOG_LVL_DEBUG, "RequestFinished " << type << " " << keepAlive.use_count()); -} - -bool SsiRequest::reply(char const* buf, int bufLen) { - Status s = SetResponse(buf, bufLen); - if (s != XrdSsiResponder::wasPosted) { - LOGS(_log, LOG_LVL_ERROR, "DANGER: Couldn't post response of length=" << bufLen); - return false; - } - return true; -} - -bool SsiRequest::replyError(std::string const& msg, int code) { - Status s = SetErrResponse(msg.c_str(), code); - if (s != XrdSsiResponder::wasPosted) { - LOGS(_log, LOG_LVL_ERROR, "DANGER: Couldn't post error response " << msg); - return false; - } - return true; -} - -bool SsiRequest::replyStream(StreamBuffer::Ptr const& sBuf, bool last) { - LOGS(_log, LOG_LVL_DEBUG, "replyStream, checking stream size=" << sBuf->getSize() << " last=" << last); - - // Normally, XrdSsi would call Recycle() when it is done with sBuf, but if this function - // returns false, then it must call Recycle(). Otherwise, the scheduler will likely - // wedge waiting for the buffer to be released. - std::lock_guard finLock(_finMutex); - if (_reqFinished) { - // Finished() was called, give up. - LOGS(_log, LOG_LVL_ERROR, "replyStream called after reqFinished."); - sBuf->Recycle(); - return false; - } - // Create a stream if needed. - if (!_stream) { - _stream = std::make_shared(); - if (SetResponse(_stream.get()) != XrdSsiResponder::Status::wasPosted) { - LOGS(_log, LOG_LVL_WARN, "SetResponse stream failed, calling Recycle for sBuf"); - // SetResponse return value indicates XrdSsi wont call Recycle(). - sBuf->Recycle(); - return false; - } - } else if (_stream->closed()) { - // XrdSsi isn't going to call Recycle if we wind up here. - LOGS(_log, LOG_LVL_ERROR, "Logic error SsiRequest::replyStream called with stream closed."); - sBuf->Recycle(); - return false; - } - // XrdSsi or Finished() will call Recycle(). - LOGS(_log, LOG_LVL_INFO, "SsiRequest::replyStream seq=" << getSeq()); - _stream->append(sBuf, last); - return true; -} - -bool SsiRequest::sendMetadata(const char* buf, int blen) { - Status stat = SetMetadata(buf, blen); - switch (stat) { - case XrdSsiResponder::wasPosted: - return true; - case XrdSsiResponder::notActive: - LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " notActive"); - break; - case XrdSsiResponder::notPosted: - LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " notPosted blen=" << blen); - break; - default: - LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " unkown state blen=" << blen); - } - return false; -} - -SsiRequest::Ptr SsiRequest::freeSelfKeepAlive() { - Ptr keepAlive = std::move(_selfKeepAlive); - return keepAlive; -} - -uint64_t SsiRequest::getSeq() const { - if (_stream == nullptr) return 0; - return _stream->getSeq(); -} - -} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/SsiRequest.h b/src/xrdsvc/SsiRequest.h deleted file mode 100644 index 3583a0cef9..0000000000 --- a/src/xrdsvc/SsiRequest.h +++ /dev/null @@ -1,128 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDSVC_SSIREQUEST_H -#define LSST_QSERV_XRDSVC_SSIREQUEST_H - -// System headers -#include -#include -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiResponder.hh" - -// Qserv headers -#include "global/ResourceUnit.h" -#include "mysql/MySqlConfig.h" -#include "xrdsvc/StreamBuffer.h" - -// Forward declarations -class XrdSsiService; - -namespace lsst::qserv { -namespace wbase { -class FileChannelShared; -class SendChannel; -class Task; -} // namespace wbase -namespace wcontrol { -class Foreman; -} -} // namespace lsst::qserv - -namespace lsst::qserv::xrdsvc { - -class ChannelStream; -class StreamBuffer; - -/// An implementation of XrdSsiResponder that is used by SsiService to provide -/// qserv worker services. The SSI interface encourages such an approach, and -/// object lifetimes are explicitly stated in the documentation which we -/// adhere to using BindRequest() and UnBindRequest() responder methods. -class SsiRequest : public XrdSsiResponder, public std::enable_shared_from_this { -public: - // Smart pointer definitions - - typedef std::shared_ptr ValidatorPtr; - typedef std::shared_ptr Ptr; - - /// Use factory to ensure proper construction for enable_shared_from_this. - static SsiRequest::Ptr newSsiRequest(std::string const& rname, - std::shared_ptr const& processor); - - virtual ~SsiRequest(); - - void execute(XrdSsiRequest& req); - - /** - * Implements the virtual method defined in the base class - * @see XrdSsiResponder::Finished - */ - void Finished(XrdSsiRequest& req, XrdSsiRespInfo const& rinfo, bool cancel = false) override; - - bool isFinished() { return _reqFinished; } - - bool reply(char const* buf, int bufLen); - bool replyError(std::string const& msg, int code); - bool replyStream(StreamBuffer::Ptr const& sbuf, bool last); - - bool sendMetadata(const char* buf, int blen); - - /// Call this to allow object to die after it truly is no longer needed. - /// i.e. It is know Finish() will not be called. - /// NOTE: It is important that any non-static SsiRequest member - /// function make a local copy of the returned pointer so that - /// SsiRequest is guaranteed to live to the end of - /// the function call. - Ptr freeSelfKeepAlive(); - - uint64_t getSeq() const; - -private: - /// Constructor (called by the static factory method newSsiRequest) - SsiRequest(std::string const& rname, std::shared_ptr const& processor); - - /// For internal error reporting - void reportError(std::string const& errStr); - -private: - ValidatorPtr _validator; ///< validates request against what's available - std::shared_ptr const _foreman; ///< actual msg processor - - std::mutex _finMutex; ///< Protects execute() from Finish(), _finished, and _stream - std::atomic _reqFinished{false}; ///< set to true when Finished called - std::string _resourceName; ///< chunk identifier - - std::shared_ptr _stream; - std::shared_ptr _channelShared; ///< Must live before Finished() gets called. - std::vector> _tasks; ///< List of tasks for use in cancellation. - - /// Make sure this object exists until Finish() is called. - /// Make a local copy before calling reset() within and non-static member function. - Ptr _selfKeepAlive; -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDSVC_SSIREQUEST_H diff --git a/src/xrdsvc/SsiService.cc b/src/xrdsvc/SsiService.cc deleted file mode 100644 index 72d484deb6..0000000000 --- a/src/xrdsvc/SsiService.cc +++ /dev/null @@ -1,270 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "xrdsvc/SsiService.h" - -// System headers -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// Third-party headers -#include -#include "XrdSsi/XrdSsiLogger.hh" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "http/Client.h" -#include "http/MetaModule.h" -#include "http/Method.h" -#include "mysql/MySqlConfig.h" -#include "mysql/MySqlConnection.h" -#include "qhttp/Server.h" -#include "sql/SqlConnection.h" -#include "sql/SqlConnectionFactory.h" -#include "util/common.h" -#include "util/FileMonitor.h" -#include "util/HoldTrack.h" -#include "wbase/Base.h" -#include "wbase/FileChannelShared.h" -#include "wconfig/WorkerConfig.h" -#include "wconfig/WorkerConfigError.h" -#include "wcontrol/Foreman.h" -#include "wcontrol/SqlConnMgr.h" -#include "wpublish/ChunkInventory.h" -#include "wsched/BlendScheduler.h" -#include "wsched/FifoScheduler.h" -#include "wsched/GroupScheduler.h" -#include "wsched/ScanScheduler.h" -#include "xrdsvc/HttpSvc.h" -#include "xrdsvc/SsiRequest.h" -#include "xrdsvc/XrdName.h" - -using namespace lsst::qserv; -using namespace nlohmann; -using namespace std; -using namespace std::literals; - -class XrdPosixCallBack; // Forward. - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiService"); - -// add LWP to MDC in log messages -void initMDC() { LOG_MDC("LWP", to_string(lsst::log::lwpID())); } -int dummyInitMDC = LOG_MDC_INIT(initMDC); - -std::shared_ptr makeChunkInventory(mysql::MySqlConfig const& mySqlConfig) { - xrdsvc::XrdName x; - if (!mySqlConfig.dbName.empty()) { - LOGS(_log, LOG_LVL_FATAL, "dbName must be empty to prevent accidental context"); - throw runtime_error("dbName must be empty to prevent accidental context"); - } - auto conn = sql::SqlConnectionFactory::make(mySqlConfig); - assert(conn); - auto inventory = make_shared(x.getName(), conn); - ostringstream os; - os << "Paths exported: "; - inventory->dbgPrint(os); - LOGS(_log, LOG_LVL_DEBUG, os.str()); - return inventory; -} - -/** - * This function will keep periodically updating worker's info in the Replication - * System's Registry. - * @param id The unique identifier of a worker to be registered. - * @note The thread will terminate the process if the registraton request to the Registry - * was explicitly denied by the service. This means the application may be misconfigured. - * Transient communication errors when attempting to connect or send requests to - * the Registry will be posted into the log stream and ignored. - */ -void registryUpdateLoop(string const& id) { - auto const workerConfig = wconfig::WorkerConfig::instance(); - auto const method = http::Method::POST; - string const url = "http://" + workerConfig->replicationRegistryHost() + ":" + - to_string(workerConfig->replicationRegistryPort()) + "/qserv-worker"; - vector const headers = {"Content-Type: application/json"}; - json const request = json::object({{"version", http::MetaModule::version}, - {"instance_id", workerConfig->replicationInstanceId()}, - {"auth_key", workerConfig->replicationAuthKey()}, - {"worker", - {{"name", id}, - {"management-port", workerConfig->replicationHttpPort()}, - {"management-host-name", util::get_current_host_fqdn()}}}}); - string const requestContext = - "SsiService: '" + http::method2string(method) + "' request to '" + url + "'"; - http::Client client(method, url, request.dump(), headers); - while (true) { - try { - json const response = client.readAsJson(); - if (0 == response.at("success").get()) { - string const error = response.at("error").get(); - LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); - abort(); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); - } - this_thread::sleep_for(chrono::seconds(max(1U, workerConfig->replicationRegistryHearbeatIvalSec()))); - } -} - -} // namespace - -namespace lsst::qserv::xrdsvc { - -SsiService::SsiService(XrdSsiLogger* log) { - LOGS(_log, LOG_LVL_DEBUG, "SsiService starting..."); - - util::HoldTrack::setup(10min); - - auto const mySqlConfig = wconfig::WorkerConfig::instance()->getMySqlConfig(); - if (not mysql::MySqlConnection::checkConnection(mySqlConfig)) { - LOGS(_log, LOG_LVL_FATAL, "Unable to connect to MySQL using configuration:" << mySqlConfig); - throw wconfig::WorkerConfigError("Unable to connect to MySQL"); - } - auto const workerConfig = wconfig::WorkerConfig::instance(); - - // Set thread pool size. - unsigned int poolSize = ranges::max({wsched::BlendScheduler::getMinPoolSize(), - workerConfig->getThreadPoolSize(), thread::hardware_concurrency()}); - - unsigned int maxPoolThreads = max(workerConfig->getMaxPoolThreads(), poolSize); - - // poolSize should be greater than either GroupScheduler::maxThreads or ScanScheduler::maxThreads - unsigned int maxThread = poolSize; - int maxReserve = 2; - auto group = make_shared("SchedGroup", maxThread, maxReserve, - workerConfig->getMaxGroupSize(), - wsched::SchedulerBase::getMaxPriority()); - - int const fastest = lsst::qserv::proto::ScanInfo::Rating::FASTEST; - int const fast = lsst::qserv::proto::ScanInfo::Rating::FAST; - int const medium = lsst::qserv::proto::ScanInfo::Rating::MEDIUM; - int const slow = lsst::qserv::proto::ScanInfo::Rating::SLOW; - int const slowest = lsst::qserv::proto::ScanInfo::Rating::SLOWEST; - double fastScanMaxMinutes = (double)workerConfig->getScanMaxMinutesFast(); - double medScanMaxMinutes = (double)workerConfig->getScanMaxMinutesMed(); - double slowScanMaxMinutes = (double)workerConfig->getScanMaxMinutesSlow(); - double snailScanMaxMinutes = (double)workerConfig->getScanMaxMinutesSnail(); - int maxTasksBootedPerUserQuery = workerConfig->getMaxTasksBootedPerUserQuery(); - int maxConcurrentBootedTasks = workerConfig->getMaxConcurrentBootedTasks(); - vector scanSchedulers{ - make_shared("SchedSlow", maxThread, workerConfig->getMaxReserveSlow(), - workerConfig->getPrioritySlow(), - workerConfig->getMaxActiveChunksSlow(), medium + 1, slow, - slowScanMaxMinutes), - make_shared("SchedFast", maxThread, workerConfig->getMaxReserveFast(), - workerConfig->getPriorityFast(), - workerConfig->getMaxActiveChunksFast(), fastest, fast, - fastScanMaxMinutes), - make_shared( - "SchedMed", maxThread, workerConfig->getMaxReserveMed(), workerConfig->getPriorityMed(), - workerConfig->getMaxActiveChunksMed(), fast + 1, medium, medScanMaxMinutes), - }; - - auto snail = make_shared( - "SchedSnail", maxThread, workerConfig->getMaxReserveSnail(), workerConfig->getPrioritySnail(), - workerConfig->getMaxActiveChunksSnail(), slow + 1, slowest, snailScanMaxMinutes); - - wpublish::QueriesAndChunks::Ptr queries = wpublish::QueriesAndChunks::setupGlobal( - chrono::minutes(5), chrono::minutes(2), maxTasksBootedPerUserQuery, maxConcurrentBootedTasks, - false); - wsched::BlendScheduler::Ptr blendSched = make_shared( - "BlendSched", queries, maxThread, group, snail, scanSchedulers); - blendSched->setPrioritizeByInFlight(false); // TODO: set in configuration file. - queries->setBlendScheduler(blendSched); - - unsigned int requiredTasksCompleted = workerConfig->getRequiredTasksCompleted(); - queries->setRequiredTasksCompleted(requiredTasksCompleted); - - int const maxSqlConn = workerConfig->getMaxSqlConnections(); - int const resvInteractiveSqlConn = workerConfig->getReservedInteractiveSqlConnections(); - auto sqlConnMgr = make_shared(maxSqlConn, maxSqlConn - resvInteractiveSqlConn); - LOGS(_log, LOG_LVL_WARN, "config sqlConnMgr" << *sqlConnMgr); - LOGS(_log, LOG_LVL_WARN, "maxPoolThreads=" << maxPoolThreads); - - _foreman = make_shared(blendSched, poolSize, maxPoolThreads, mySqlConfig, queries, - ::makeChunkInventory(mySqlConfig), sqlConnMgr); - - // Watch to see if the log configuration is changed. - // If LSST_LOG_CONFIG is not defined, there's no good way to know what log - // configuration file is in use. - string logConfigFile = std::getenv("LSST_LOG_CONFIG"); - if (logConfigFile == "") { - LOGS(_log, LOG_LVL_ERROR, - "FileMonitor LSST_LOG_CONFIG was blank, no log configuration file to watch."); - } else { - LOGS(_log, LOG_LVL_ERROR, "logConfigFile=" << logConfigFile); - _logFileMonitor = make_shared(logConfigFile); - } - - // Garbage collect unclaimed result files (if any). - // ATTENTION: this is the blocking operation since it needs to be run before accepting - // new queries to ensure that worker had sufficient resources to process those. - if (workerConfig->resultsCleanUpOnStart()) { - wbase::FileChannelShared::cleanUpResultsOnWorkerRestart(); - } - - // Start the control server for processing worker management requests sent - // by the Replication System. Update the port number in the configuration - // in case if the server is run on the dynamically allocated port. - _controlHttpSvc = HttpSvc::create(_foreman, workerConfig->replicationHttpPort(), - workerConfig->replicationNumHttpThreads()); - auto const port = _controlHttpSvc->start(); - workerConfig->setReplicationHttpPort(port); - - // Begin periodically updating worker's status in the Replication System's registry - // in the detached thread. This will continue before the application gets terminated. - thread registryUpdateThread(::registryUpdateLoop, _foreman->chunkInventory()->id()); - registryUpdateThread.detach(); -} - -SsiService::~SsiService() { - LOGS(_log, LOG_LVL_DEBUG, "SsiService dying."); - _controlHttpSvc->stop(); -} - -void SsiService::ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) { - LOGS(_log, LOG_LVL_DEBUG, "Got request call where rName is: " << resRef.rName); - auto request = SsiRequest::newSsiRequest(resRef.rName, _foreman); - - // Continue execution in the session object as SSI gave us a new thread. - // Object deletes itself when finished is called. - // - request->execute(reqRef); -} - -} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/StreamBuffer.cc b/src/xrdsvc/StreamBuffer.cc deleted file mode 100644 index 8024aa5684..0000000000 --- a/src/xrdsvc/StreamBuffer.cc +++ /dev/null @@ -1,139 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "xrdsvc/StreamBuffer.h" - -// Third-party headers -#include "boost/utility.hpp" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "wbase/Task.h" -#include "wcontrol/WorkerStats.h" - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.StreamBuffer"); -} - -using namespace std; - -namespace lsst::qserv::xrdsvc { - -// Factory function, because this should be able to delete itself when Recycle() is called. -StreamBuffer::Ptr StreamBuffer::createWithMove(std::string &input, std::shared_ptr const &task) { - Ptr ptr(new StreamBuffer(input, task)); - ptr->_selfKeepAlive = ptr; - return ptr; -} - -StreamBuffer::StreamBuffer(std::string &input, wbase::Task::Ptr const &task) : _task(task) { - _dataStr = std::move(input); - // TODO: try to make 'data' a const char* in xrootd code. - // 'data' is not being changed after being passed, so hopefully not an issue. - //_dataStr will not be used again, but this is ugly. - data = (char *)(_dataStr.data()); - next = 0; - - auto now = CLOCK::now(); - _createdTime = now; - _startTime = now; - _endTime = now; - - _wStats = wcontrol::WorkerStats::get(); - if (_wStats != nullptr) { - _wStats->startQueryRespConcurrentQueued(_createdTime); - } -} - -void StreamBuffer::startTimer() { - auto now = CLOCK::now(); - _startTime = now; - _endTime = now; - - if (_wStats != nullptr) { - _wStats->endQueryRespConcurrentQueued(_createdTime, _startTime); // add time to queued time - } -} - -/// xrdssi calls this to recycle the buffer when finished. -void StreamBuffer::Recycle() { - { - std::lock_guard lg(_mtx); - _doneWithThis = true; - } - _cv.notify_all(); - - _endTime = CLOCK::now(); - if (_wStats != nullptr) { - _wStats->endQueryRespConcurrentXrootd(_startTime, _endTime); - } - - if (_task != nullptr) { - auto taskSched = _task->getTaskScheduler(); - if (taskSched != nullptr) { - std::chrono::duration secs = _endTime - _startTime; - taskSched->histTimeOfTransmittingTasks->addEntry(secs.count()); - LOGS(_log, LOG_LVL_TRACE, "Recycle " << taskSched->histTimeOfTransmittingTasks->getJson()); - } else { - LOGS(_log, LOG_LVL_WARN, "Recycle transmit taskSched == nullptr"); - } - } else { - LOGS(_log, LOG_LVL_DEBUG, "Recycle transmit _task == nullptr"); - } - // Effectively reset _selfKeepAlive, and if nobody else was - // referencing this, this object will delete itself when - // this function is done. - // std::move is used instead of reset() as reset() could - // result in _keepalive deleting itself while still in use. - Ptr keepAlive = std::move(_selfKeepAlive); -} - -void StreamBuffer::cancel() { - // Recycle may still need to be called by XrdSsi or there will be a memory - // leak. XrdSsi calling Recycle is beyond what can be controlled here, but - // better a possible leak than corrupted memory or a permanently wedged - // thread in a limited pool. - // In any case, this code having an effect should be extremely rare. - // FUTURE: It would be nice to eliminate this possible memory leak. - // Possible fix, atomic _recycleCalled, create thread - // to check if _recycleCalled == true. If true or 24 hours pass - // use `Ptr keepAlive = std::move(_selfKeepAlive);` to kill the object. - { - std::lock_guard lg(_mtx); - _doneWithThis = true; - _cancelled = true; - } - _cv.notify_all(); -} - -// Wait until recycle is called. -bool StreamBuffer::waitForDoneWithThis() { - std::unique_lock uLock(_mtx); - _cv.wait(uLock, [this]() { return _doneWithThis || _cancelled; }); - return !_cancelled; -} - -} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/StreamBuffer.h b/src/xrdsvc/StreamBuffer.h deleted file mode 100644 index f704e2c70a..0000000000 --- a/src/xrdsvc/StreamBuffer.h +++ /dev/null @@ -1,113 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDSVC_STREAMBUFFER_H -#define LSST_QSERV_XRDSVC_STREAMBUFFER_H - -// System headers -#include -#include -#include -#include -#include - -// qserv headers -#include "util/InstanceCount.h" - -// Third-party headers -#include "XrdSsi/XrdSsiErrInfo.hh" // required by XrdSsiStream -#include "XrdSsi/XrdSsiStream.hh" - -namespace lsst::qserv { -namespace wbase { -class Task; -} -namespace wcontrol { -class WorkerStats; -} -} // namespace lsst::qserv - -namespace lsst::qserv::xrdsvc { - -/// StreamBuffer is a single use buffer for transferring data packets -/// to XrdSsi. -/// Its notable feature is the Recycle() function, which XrdSsi will -/// promptly call when it no longer needs the buffer. -class StreamBuffer : public XrdSsiStream::Buffer { -public: - using Ptr = std::shared_ptr; - - // Copying this would be very confusing for something waiting for Recycle(). - StreamBuffer() = delete; - StreamBuffer(StreamBuffer const &) = delete; - StreamBuffer &operator=(StreamBuffer const &) = delete; - - /// Factory function, because this should be able to delete itself when Recycle() is called. - /// The constructor uses move to avoid copying the string. - static StreamBuffer::Ptr createWithMove(std::string &input, - std::shared_ptr const &task = nullptr); - - size_t getSize() const { return _dataStr.size(); } - - /// Call to recycle the buffer when finished (normally called by XrdSsi). - void Recycle() override; - - /// Wait until Recycle() is called. - /// @return true if there is data in the buffer. - bool waitForDoneWithThis(); - - /// Start the timer that will be stopped when Recycle() is called. - void startTimer(); - - /// Unblock the condition variable on cancel. - void cancel(); - - ~StreamBuffer() override = default; - -private: - /// This constructor will invalidate 'input'. - explicit StreamBuffer(std::string &input, std::shared_ptr const &task); - - /// Pointer to the task for keeping statistics. - /// NOTE: This will be nullptr for many things, so check before using. - std::shared_ptr _task; - std::string _dataStr; - std::mutex _mtx; - std::condition_variable _cv; - bool _doneWithThis = false; - bool _cancelled = false; - Ptr _selfKeepAlive; ///< keep this object alive until after Recycle() is called. - // util::InstanceCount _ic{"StreamBuffer"}; ///< Useful as it indicates amount of waiting for czar. - - std::chrono::time_point _createdTime; ///< Time this instance was created. - std::chrono::time_point - _startTime; ///< Time this instance was handed to xrootd. - std::chrono::time_point - _endTime; ///< Time xrootd was finished with this instance. - /// Pointer for worker statistics. - /// NOTE: This will be nullptr for many things, so check before using. - std::shared_ptr _wStats; -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDSVC_STREAMBUFFER_H From 83fb1fd0c9f97f702a3d0503d5dc2afe05de5ae5 Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 18 Dec 2024 09:37:50 -0800 Subject: [PATCH 03/15] Changed Czar to catch 5GB limit. --- .../migrations/czar/migrate-10-to-11.sql | 15 + src/cconfig/CzarConfig.h | 52 +--- src/ccontrol/MergingHandler.cc | 192 ++---------- src/ccontrol/MergingHandler.h | 5 - src/ccontrol/UserQueryFactory.cc | 7 +- src/ccontrol/UserQueryFactory.h | 2 +- src/ccontrol/UserQueryQueries.cc | 4 +- src/ccontrol/UserQueryQueries.h | 4 +- src/ccontrol/UserQuerySelect.cc | 144 +++++---- src/czar/ActiveWorker.cc | 2 +- src/czar/ActiveWorker.h | 7 +- src/czar/Czar.cc | 31 +- src/czar/CzarChunkMap.cc | 26 +- src/czar/CzarChunkMap.h | 5 +- src/czar/HttpCzarWorkerModule.cc | 15 +- src/czar/HttpMonitorModule.cc | 2 +- src/czar/testCzar.cc | 4 +- src/http/BaseModule.cc | 4 +- src/http/BaseModule.h | 4 + src/protojson/UberJobMsg.cc | 31 +- src/protojson/UberJobMsg.h | 54 ++-- src/protojson/WorkerQueryStatusData.h | 2 +- src/protojson/testUberJobMsg.cc | 11 +- src/qana/QueryMapping.h | 2 - src/qdisp/Executive.cc | 120 +++++--- src/qdisp/Executive.h | 53 ++-- src/qdisp/JobDescription.cc | 22 +- src/qdisp/JobDescription.h | 7 +- src/qdisp/JobQuery.cc | 12 +- src/qdisp/JobQuery.h | 1 - src/qdisp/UberJob.cc | 109 +++---- src/qdisp/UberJob.h | 14 +- src/qdisp/testQDisp.cc | 26 +- src/qhttp/Server.cc | 37 ++- src/qmeta/QMetaMysql.cc | 1 + src/qproc/ChunkQuerySpec.h | 5 +- src/qproc/QuerySession.cc | 1 - src/replica/contr/HttpQservMonitorModule.cc | 8 +- src/rproc/InfileMerger.cc | 100 +----- src/rproc/InfileMerger.h | 4 - src/util/Error.cc | 6 +- src/util/Error.h | 4 +- src/util/InstanceCount.cc | 14 +- src/util/Mutex.h | 2 +- src/util/QdispPool.cc | 3 +- src/util/ResultFileNameParser.cc | 79 +++++ src/util/ThreadPool.h | 7 +- src/util/common.cc | 23 ++ src/util/common.h | 6 + src/util/testFileNameParser.cc | 110 +++++++ src/wbase/FileChannelShared.cc | 32 +- src/wbase/FileChannelShared.h | 6 +- src/wbase/Task.cc | 257 ++++------------ src/wbase/Task.h | 73 ++--- src/wbase/UberJobData.cc | 91 ++++-- src/wbase/UberJobData.h | 52 +++- src/wconfig/WorkerConfig.h | 24 +- src/wcontrol/Foreman.cc | 26 +- src/wcontrol/Foreman.h | 23 +- src/wcontrol/SqlConnMgr.cc | 6 +- src/wcontrol/WCzarInfoMap.cc | 7 +- src/wdb/ChunkResource.cc | 12 +- src/wdb/QueryRunner.cc | 45 ++- src/wdb/QueryRunner.h | 3 +- src/wdb/testQueryRunner.cc | 26 +- src/wpublish/QueriesAndChunks.cc | 21 +- src/wpublish/QueriesAndChunks.h | 15 +- src/wsched/BlendScheduler.cc | 125 ++++---- src/wsched/BlendScheduler.h | 12 +- src/wsched/ChunkTasksQueue.cc | 43 +-- src/wsched/ChunkTasksQueue.h | 2 +- src/wsched/ScanScheduler.cc | 31 +- src/wsched/SchedulerBase.cc | 28 +- src/wsched/SchedulerBase.h | 24 +- src/xrdsvc/HttpModule.cc | 7 +- src/xrdsvc/HttpSvc.cc | 3 +- src/xrdsvc/HttpWorkerCzarModule.cc | 86 ++++-- src/xrdsvc/HttpWorkerCzarModule.h | 18 ++ src/xrdsvc/SsiService.cc | 291 ++++++++++++++++++ 79 files changed, 1592 insertions(+), 1196 deletions(-) create mode 100644 src/util/ResultFileNameParser.cc create mode 100644 src/util/testFileNameParser.cc create mode 100644 src/xrdsvc/SsiService.cc diff --git a/python/lsst/qserv/schema/migrations/czar/migrate-10-to-11.sql b/python/lsst/qserv/schema/migrations/czar/migrate-10-to-11.sql index 854f5624e6..b386806170 100644 --- a/python/lsst/qserv/schema/migrations/czar/migrate-10-to-11.sql +++ b/python/lsst/qserv/schema/migrations/czar/migrate-10-to-11.sql @@ -34,3 +34,18 @@ CREATE OR REPLACE LEFT OUTER JOIN `QTable` AS `qt` ON `qi`.`queryId`=`qt`.`queryId` LEFT OUTER JOIN `QMessages` AS `qm` ON `qi`.`queryId`=`qm`.`queryId` GROUP BY `qi`.`queryId`; + + +CREATE TABLE IF NOT EXISTS `chunkMap` ( + `worker` VARCHAR(256) NOT NULL COMMENT 'A unique identifier of a worker hosting the chunk replica', + `database` VARCHAR(256) NOT NULL COMMENT 'The name of a database', + `table` VARCHAR(256) NOT NULL COMMENT 'The name of a table', + `chunk` INT UNSIGNED NOT NULL COMMENT 'The number of a chunk', + `size` BIGINT UNSIGNED NOT NULL COMMENT 'The size of a chunk') +ENGINE = InnoDB +COMMENT = 'Chunk disposition across workers'; + +CREATE TABLE IF NOT EXISTS `chunkMapStatus` ( + `update_time` TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'The most recent update time of the map') +ENGINE = InnoDB +COMMENT = 'Satus info on the chunk map'; diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 40aab039b4..89964b0bbe 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -115,36 +115,8 @@ class CzarConfig { */ int getInteractiveChunkLimit() const { return _interactiveChunkLimit->getVal(); } - /* Get hostname and port for xrootd manager - * - * "localhost:1094" is the most reasonable default, even though it is - * the wrong choice for all but small developer installations - * - * @return a string containing ":" - */ - std::string const& getXrootdFrontendUrl() const { return _xrootdFrontendUrl->getVal(); } - - /* Get the maximum number of threads for xrootd to use. - * - * @return the maximum number of threads for xrootd to use. - */ - int getXrootdCBThreadsMax() const { return _xrootdCBThreadsMax->getVal(); } - - /* Get the initial number of threads for xrootd to create and maintain. - * - * @return the initial number of threads for xrootd to use. - */ - int getXrootdCBThreadsInit() const { return _xrootdCBThreadsInit->getVal(); } - bool getQueryDistributionTestVer() const { return _queryDistributionTestVer->getVal(); } - /* - * @return A value of the "spread" parameter. This may improve a performance - * of xrootd for catalogs with the large number of chunks. The default value - * of this parameter in xrootd is 4. - */ - int getXrootdSpread() const { return _xrootdSpread->getVal(); } - /* Get minimum number of seconds between QMeta chunk completion updates. * * @return seconds between QMeta chunk completion updates. @@ -226,6 +198,9 @@ class CzarConfig { /// Return the sleep time (in milliseconds) between messages sent to active workers. int getMonitorSleepTimeMilliSec() const { return _monitorSleepTimeMilliSec->getVal(); } + /// Return true if family map chunk distribution should depend on chunk size. + bool getFamilyMapUsingChunkSize() const { return _familyMapUsingChunkSize->getVal(); } + // Parameters of the Czar management service std::string const& replicationInstanceId() const { return _replicationInstanceId->getVal(); } @@ -366,8 +341,6 @@ class CzarConfig { CVTStrPtr _qstatusDb = util::ConfigValTStr::create(_configValMap, "qstatus", "db", notReq, "qservStatusData"); - CVTStrPtr _xrootdFrontendUrl = - util::ConfigValTStr::create(_configValMap, "frontend", "xrootd", notReq, "localhost:1094"); CVTStrPtr _emptyChunkPath = util::ConfigValTStr::create(_configValMap, "partitioner", "emptyChunkPath", notReq, "."); CVTIntPtr _maxMsgSourceStore = @@ -385,15 +358,14 @@ class CzarConfig { CVTStrPtr _qdispVectMinRunningSizes = util::ConfigValTStr::create(_configValMap, "qdisppool", "vectMinRunningSizes", notReq, "0:3:3:3"); - CVTIntPtr _xrootdSpread = util::ConfigValTInt::create(_configValMap, "tuning", "xrootdSpread", notReq, 4); + // UberJobs + CVTIntPtr _uberJobMaxChunks = + util::ConfigValTInt::create(_configValMap, "uberjob", "maxChunks", notReq, 10000); + CVTIntPtr _qMetaSecsBetweenChunkCompletionUpdates = util::ConfigValTInt::create( _configValMap, "tuning", "qMetaSecsBetweenChunkCompletionUpdates", notReq, 60); CVTIntPtr _interactiveChunkLimit = util::ConfigValTInt::create(_configValMap, "tuning", "interactiveChunkLimit", notReq, 10); - CVTIntPtr _xrootdCBThreadsMax = - util::ConfigValTInt::create(_configValMap, "tuning", "xrootdCBThreadsMax", notReq, 500); - CVTIntPtr _xrootdCBThreadsInit = - util::ConfigValTInt::create(_configValMap, "tuning", "xrootdCBThreadsInit", notReq, 50); CVTIntPtr _queryDistributionTestVer = util::ConfigValTInt::create(_configValMap, "tuning", "queryDistributionTestVer", notReq, 0); CVTBoolPtr _notifyWorkersOnQueryFinish = @@ -433,12 +405,12 @@ class CzarConfig { util::ConfigValTInt::create(_configValMap, "activeworker", "timeoutDeadSecs", notReq, 60 * 10); CVTIntPtr _activeWorkerMaxLifetimeSecs = // 1hr util::ConfigValTInt::create(_configValMap, "activeworker", "maxLifetimeSecs", notReq, 60 * 60); - CVTIntPtr _monitorSleepTimeMilliSec = - util::ConfigValTInt::create(_configValMap, "activeworker", "monitorSleepTimeMilliSec", notReq, 15'000); + CVTIntPtr _monitorSleepTimeMilliSec = util::ConfigValTInt::create( + _configValMap, "activeworker", "monitorSleepTimeMilliSec", notReq, 15'000); - // UberJobs - CVTIntPtr _uberJobMaxChunks = - util::ConfigValTInt::create(_configValMap, "uberjob", "maxChunks", notReq, 1000); + // FamilyMap + CVTBoolPtr _familyMapUsingChunkSize = + util::ConfigValTBool::create(_configValMap, "familymap", "usingChunkSize", notReq, 0); /// This may impact `_resultMaxHttpConnections` as too many connections may cause kernel memory issues. CVTIntPtr _commandMaxHttpConnections = diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index cc0da0f783..18e2785dce 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -87,79 +87,12 @@ lsst::qserv::TimeCountTracker::CALLBACKFUNC const reportFileRecvRate = } }; -/** - * This exception is used by the merging handler to signal the file reader - * that the query has been ended before the file has been completely read. - * The exception is meant to tell the reader to stop reading the file - * and return control to the caller. - */ -class QueryEnded : public std::runtime_error { -public: - using std::runtime_error::runtime_error; -}; - -/** - * The function for reading result files from workers over the HTTP protocol. - * The function reads the file in chunks and calls the callback function - * for each chunk of data read from the file. - */ -string readHttpFileAndMerge(string const& httpUrl, size_t fileSize, - function const& messageIsReady) { - string const context = "MergingHandler::" + string(__func__) + " "; - - LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); - - // Track the file while the control flow is staying within the function. - ResultFileTracker const resultFileTracker; - - // The data transmit rate tracker is set up before reading each data message. - unique_ptr> transmitRateTracker; - - // A location of the next byte to be read from the input file. The variable - // is used for error reporting. - size_t offset = 0; - - try { - string const noClientData; - vector const noClientHeaders; - - http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, - qdisp::QueryRequest::makeHttpClientConfig(), - qdisp::QueryRequest::getHttpConnPool()); - - // Starts the tracker to measure the performance of the network I/O. - transmitRateTracker = make_unique>(reportFileRecvRate); - - // Start reading the file. The read() method will call the callback function - // for each chunk of data read from the file. - reader.read([&](char const* inBuf, size_t inBufSize) { - // Check if the end of the file has been reached. - // Destroying the tracker will result in stopping the tracker's timer and - // reporting the file read rate before proceeding to the merge. - transmitRateTracker->addToValue(inBufSize); - transmitRateTracker->setSuccess(); - transmitRateTracker.reset(); - messageIsReady(inBuf, inBufSize); - offset += inBufSize; - // Restart the tracker to measure the reading performance of the next chunk of data. - transmitRateTracker = make_unique>(reportFileRecvRate); - }); - if (offset != fileSize) { - throw runtime_error(context + "short read"); - } - } catch (QueryEnded const& ex) { - // This is a normal condition which should be handled gracefully by the algorithm. - LOGS(_log, LOG_LVL_DEBUG, context << ex.what() << ", httpUrl=" << httpUrl); - } catch (exception const& ex) { - string const errMsg = "failed to open/read: " + httpUrl + ", fileSize: " + to_string(fileSize) + - ", offset: " + to_string(offset) + ", ex: " + string(ex.what()); - LOGS(_log, LOG_LVL_ERROR, context << errMsg); - return errMsg; - } - return string(); -} - +/// If success, then everything is fine. +/// If not success, and not mergeHappened, the user query can be saved by abandoning +/// this UberJob. If mergeHappened, the result table is fouled and the user query is ruined. +/// @return bool success - true if operation was successful. +/// @return bool mergeHappened - true if merging was started. std::tuple readHttpFileAndMergeHttp( lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, function const& messageIsReady, @@ -193,10 +126,14 @@ std::tuple readHttpFileAndMergeHttp( // The value is stays 0 while reading the frame header. uint32_t msgSizeBytes = 0; bool success = true; - bool mergeSuccess = true; + bool mergeHappened = false; int headerCount = 0; uint64_t totalBytesRead = 0; try { + auto exec = uberJob->getExecutive(); + if (exec == nullptr || exec->getCancelled()) { + throw runtime_error(context + " query was cancelled"); + } string const noClientData; vector const noClientHeaders; http::ClientConfig clientConfig; @@ -213,10 +150,11 @@ std::tuple readHttpFileAndMergeHttp( bool last = false; char const* next = inBuf; char const* const end = inBuf + inBufSize; + LOGS(_log, LOG_LVL_TRACE, context << " next=" << (uint64_t)next << " end=" << (uint64_t)end); while ((next < end) && !last) { - LOGS(_log, LOG_LVL_WARN, - context << "TODO:UJ next=" << (uint64_t)next << " end=" << (uint64_t)end - << " last=" << last); + if (exec->getCancelled()) { + throw runtime_error(context + " query was cancelled"); + } if (msgSizeBytes == 0) { // Continue or finish reading the frame header. size_t const bytes2read = @@ -273,9 +211,10 @@ std::tuple readHttpFileAndMergeHttp( } // Parse and evaluate the message. - mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); + mergeHappened = true; + bool messageReadyResult = messageIsReady(msgBuf.get(), msgSizeBytes, last); totalBytesRead += msgSizeBytes; - if (!mergeSuccess) { + if (!messageReadyResult) { success = false; throw runtime_error("message processing failed at offset " + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); @@ -283,14 +222,14 @@ std::tuple readHttpFileAndMergeHttp( // Reset the variable to prepare for reading the next header & message (if any). msgSizeBytes = 0; } else { - LOGS(_log, LOG_LVL_WARN, + LOGS(_log, LOG_LVL_TRACE, context << " headerCount=" << headerCount << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); } } } }); - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes << " totalBytesRead=" << totalBytesRead); if (msgSizeBufNext != 0) { @@ -316,8 +255,8 @@ std::tuple readHttpFileAndMergeHttp( } // If the merge failed, that indicates something went wrong in the local database table, // is likely this user query is doomed and should be cancelled. - LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeSuccess); - return {success, mergeSuccess}; + LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeHappened); + return {success, mergeHappened}; } } // namespace @@ -325,11 +264,9 @@ std::tuple readHttpFileAndMergeHttp( namespace lsst::qserv::ccontrol { MergingHandler::MergingHandler(std::shared_ptr merger, std::string const& tableName) - : _infileMerger{merger}, _tableName{tableName} { - _initState(); -} + : _infileMerger{merger}, _tableName{tableName} {} -MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << _tableName); } +MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_TRACE, __func__ << " " << _tableName); } bool MergingHandler::flush(proto::ResponseSummary const& resp) { @@ -383,7 +320,6 @@ void MergingHandler::errorFlush(std::string const& msg, int code) { LOGS(_log, LOG_LVL_ERROR, "Error receiving result."); } - // Note that generally we always have an _infileMerger object except during // a unit test. I suppose we could try to figure out how to create one. // @@ -395,84 +331,6 @@ std::ostream& MergingHandler::print(std::ostream& os) const { return os << "MergingRequester(" << _tableName << ", flushed=" << (_flushed ? "true)" : "false)"); } -void MergingHandler::_initState() { _setError(util::ErrorCode::NONE, string()); } - -bool MergingHandler::_queryIsNoLongerActive(shared_ptr const& jobQuery) const { - // Check if the query got cancelled for any reason. - if (jobQuery->isQueryCancelled()) return true; - - // Check for other indicators that the query may have cancelled or finished. - auto executive = jobQuery->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { - return true; - } - - // The final test is to see if any errors have been reported in a context - // of the merger. A presence of errors means that further attempting of merging - // makes no sense. - return !getError().isNone(); -} - -bool MergingHandler::_merge(proto::ResponseSummary const& resp, shared_ptr const& jobQuery) { - if (_flushed) throw util::Bug(ERR_LOC, "already flushed"); - if (resp.transmitsize() == 0) return true; - - // After this final test the job's result processing can't be interrupted. - if (_queryIsNoLongerActive(jobQuery)) return true; - - // Read from the http stream and push records into the CSV stream in a separate thread. - // Note the fixed capacity of the stream which allows up to 2 records to be buffered - // in the stream. This is enough to hide the latency of the HTTP connection and - // the time needed to read the file. - auto csvStream = mysql::CsvStream::create(2); - string fileReadErrorMsg; - thread csvThread([&]() { - size_t bytesRead = 0; - fileReadErrorMsg = ::readHttpFileAndMerge( - resp.fileresource_http(), resp.transmitsize(), [&](char const* buf, uint32_t size) { - bool const queryEnded = _queryIsNoLongerActive(jobQuery); - bool last = false; - if (buf == nullptr || size == 0 || queryEnded) { - last = true; - } else { - csvStream->push(buf, size); - bytesRead += size; - last = bytesRead >= resp.transmitsize(); - } - if (last) { - csvStream->push(nullptr, 0); - if (queryEnded) { - throw ::QueryEnded( - "query " + jobQuery->getIdStr() + - " ended while reading the file, bytesRead=" + to_string(bytesRead) + - ", transmitsize=" + to_string(resp.transmitsize())); - } - } - }); - // Push the stream terminator to indicate the end of the stream. - // It may be neeeded to unblock the table merger which may be still attempting to read - // from the CSV stream. - if (!fileReadErrorMsg.empty()) { - csvStream->push(nullptr, 0); - } - }); - - // Attempt the actual merge. - bool const fileMergeSuccess = _infileMerger->merge(resp, csvStream); - if (!fileMergeSuccess) { - LOGS(_log, LOG_LVL_WARN, __func__ << " merge failed"); - util::Error const& err = _infileMerger->getError(); - _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg()); - } - csvThread.join(); - if (!fileReadErrorMsg.empty()) { - LOGS(_log, LOG_LVL_WARN, __func__ << " result file read failed"); - _setError(ccontrol::MSG_HTTP_RESULT, fileReadErrorMsg); - } - _flushed = true; - return fileMergeSuccess && fileReadErrorMsg.empty(); -} - bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, proto::ResponseData const& responseData) { if (_flushed) { @@ -488,7 +346,7 @@ bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, } void MergingHandler::_setError(int code, std::string const& msg) { - LOGS(_log, LOG_LVL_DEBUG, "_setErr: code: " << code << ", message: " << msg); + LOGS(_log, LOG_LVL_DEBUG, "_setError: code: " << code << ", message: " << msg); std::lock_guard lock(_errorMutex); _error = Error(code, msg); } @@ -535,7 +393,7 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe } if (success) { - _infileMerger->mergeCompleteFor(uberJob->getJobId()); + _infileMerger->mergeCompleteFor(uberJob->getUjId()); } return {success, shouldCancel}; } diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index b7c7343e79..fc5b7e0d37 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -92,11 +92,6 @@ class MergingHandler : public qdisp::ResponseHandler { } private: - /// Prepare for first call to flush(). - void _initState(); - - bool _merge(proto::ResponseSummary const& resp, std::shared_ptr const& jobQuery); - /// Call InfileMerger to do the work of merging this data to the result. bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 8d101bf32a..71843811e4 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -224,12 +224,9 @@ std::shared_ptr makeUserQuerySharedResources( //////////////////////////////////////////////////////////////////////// UserQueryFactory::UserQueryFactory(qproc::DatabaseModels::Ptr const& dbModels, std::string const& czarName) : _userQuerySharedResources(makeUserQuerySharedResources(dbModels, czarName)), + _qmetaSecondsBetweenUpdates(cconfig::CzarConfig::instance()->getQMetaSecondsBetweenChunkUpdates()), _useQservRowCounterOptimization(true), _asioIoService() { - auto const czarConfig = cconfig::CzarConfig::instance(); - _executiveConfig = std::make_shared( - czarConfig->getXrootdFrontendUrl(), czarConfig->getQMetaSecondsBetweenChunkUpdates()); - // When czar crashes/exits while some queries are still in flight they // are left in EXECUTING state in QMeta. We want to cleanup that state // to avoid confusion. Note that when/if clean czar restart is implemented @@ -357,7 +354,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st std::shared_ptr executive; std::shared_ptr infileMergerConfig; if (sessionValid) { - executive = qdisp::Executive::create(*_executiveConfig, messageStore, qdispSharedResources, + executive = qdisp::Executive::create(_qmetaSecondsBetweenUpdates, messageStore, qdispPool, _userQuerySharedResources->queryProgress, _userQuerySharedResources->queryProgressHistory, qs, _asioIoService); diff --git a/src/ccontrol/UserQueryFactory.h b/src/ccontrol/UserQueryFactory.h index 90e5109794..78199c60c7 100644 --- a/src/ccontrol/UserQueryFactory.h +++ b/src/ccontrol/UserQueryFactory.h @@ -92,7 +92,7 @@ class UserQueryFactory : private boost::noncopyable { private: std::shared_ptr _userQuerySharedResources; - std::shared_ptr _executiveConfig; + int _qmetaSecondsBetweenUpdates; ///< Seconds between qmeta updates. bool _useQservRowCounterOptimization; bool _debugNoMerge = false; // BOOST ASIO service is started to process asynchronous timer requests diff --git a/src/ccontrol/UserQueryQueries.cc b/src/ccontrol/UserQueryQueries.cc index 753d277ded..3a6d30e354 100644 --- a/src/ccontrol/UserQueryQueries.cc +++ b/src/ccontrol/UserQueryQueries.cc @@ -36,7 +36,7 @@ #include "cconfig/CzarConfig.h" #include "css/CssAccess.h" #include "css/CssError.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "qmeta/Exceptions.h" #include "qmeta/QMetaSelect.h" #include "query/FromList.h" @@ -68,7 +68,7 @@ UserQueryQueries::UserQueryQueries(std::shared_ptr const& sta std::string const& resultDb) : _qMetaSelect(qMetaSelect), _czarId(czarId), - _messageStore(std::make_shared()), + _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), _resultDb(resultDb) { // The SQL statement should be mostly OK alredy but we need to change diff --git a/src/ccontrol/UserQueryQueries.h b/src/ccontrol/UserQueryQueries.h index 5318ee2afa..982472d0cf 100644 --- a/src/ccontrol/UserQueryQueries.h +++ b/src/ccontrol/UserQueryQueries.h @@ -84,7 +84,7 @@ class UserQueryQueries : public UserQuery { void discard() override; // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } /// @return Name of the result table for this query, can be empty std::string getResultTableName() const override { return _resultTableName; } @@ -102,7 +102,7 @@ class UserQueryQueries : public UserQuery { std::shared_ptr _qMetaSelect; qmeta::CzarId const _czarId; ///< Czar ID in QMeta database QueryState _qState = UNKNOWN; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; std::string _resultTableName; std::string _query; ///< query to execute on QMeta database std::string _orderBy; diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index c294f82674..82a01af775 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -107,7 +107,6 @@ #include "sql/Schema.h" #include "util/Bug.h" #include "util/IterableFormatter.h" -#include "util/Histogram.h" //&&& #include "util/QdispPool.h" #include "util/ThreadPriority.h" #include "qdisp/UberJob.h" @@ -156,30 +155,26 @@ std::string UserQuerySelect::getError() const { /// Attempt to kill in progress. void UserQuerySelect::kill() { - LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect kill"); - // The lock must be held for the entire query cancellation operation to prevent - // the race condition if the query join() or discard() happens at the same time. + LOGS(_log, LOG_LVL_INFO, "UserQuerySelect KILL"); std::lock_guard lock(_killMutex); - if (_killed) return; - - // If either pointer is nullptr then it's too late for killing the query. The query - // has already been finished or it's in a process of beging finished. - if (_executive == nullptr || _infileMerger == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect kill: the query is already finished"); - return; + if (!_killed) { + _killed = true; + auto exec = _executive; + int64_t collectedRows = (exec) ? exec->getTotalResultRows() : -1; + size_t collectedBytes = _infileMerger->getTotalResultSize(); + try { + if (exec != nullptr) { + exec->squash("UserQuerySelect::kill"); + } + } catch (UserQueryError const& e) { + // Silence merger discarding errors, because this object is being + // released. Client no longer cares about merger errors. + } + // Since this is being aborted, collectedRows and collectedBytes are going to + // be off a bit as results were still coming in. A rough idea should be + // good enough. + _qMetaUpdateStatus(qmeta::QInfo::ABORTED, collectedRows, collectedBytes, 0); } - try { - _executive->squash(); - } catch (UserQueryError const& e) { - // Silence merger discarding errors, because this object is being - // released. Client no longer cares about merger errors. - } - // Since this is being aborted, collectedRows and collectedBytes are going to - // be off a bit as results were still coming in. A rough idea should be - // good enough. - _qMetaUpdateStatus(qmeta::QInfo::ABORTED, _executive->getTotalResultRows(), - _infileMerger->getTotalResultSize(), 0); - _killed = true; } std::string UserQuerySelect::_getResultOrderBy() const { return _qSession->getResultOrderBy(); } @@ -235,6 +230,11 @@ std::string UserQuerySelect::getResultQuery() const { /// Begin running on all chunks added so far. void UserQuerySelect::submit() { + auto exec = _executive; + if (exec == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "UserQuerySelect::submit() executive is null at start"); + return; + } _qSession->finalize(); // Using the QuerySession, generate query specs (text, db, chunkId) and then @@ -260,13 +260,13 @@ void UserQuerySelect::submit() { LOGS(_log, LOG_LVL_WARN, "Failed QProgress::insert, ex: " << e.what()); } - _executive->setScanInteractive(_qSession->getScanInteractive()); - _executive->setScanInfo(_qSession->getScanInfo()); + exec->setScanInteractive(_qSession->getScanInteractive()); + exec->setScanInfo(_qSession->getScanInfo()); string dbName(""); bool dbNameSet = false; - for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); + for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !exec->getCancelled(); ++i) { auto& chunkSpec = *i; @@ -294,33 +294,34 @@ void UserQuerySelect::submit() { return; } dbName = cs->db; + _queryDbName = dbName; dbNameSet = true; } ResourceUnit ru; ru.setAsDbChunk(cs->db, cs->chunkId); qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( - _qMetaCzarId, _executive->getId(), sequence, ru, + _qMetaCzarId, exec->getId(), sequence, ru, std::make_shared(_infileMerger, chunkResultName), cs, chunkResultName); - auto job = _executive->add(jobDesc); + auto job = exec->add(jobDesc); ++sequence; } - if (dbNameSet) { - _queryDbName = dbName; - } - /// At this point the executive has a map of all jobs with the chunkIds as the key. // This is needed to prevent Czar::_monitor from starting things before they are ready. - _executive->setReadyToExecute(); + exec->setAllJobsCreated(); buildAndSendUberJobs(); LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); // TODO:UJ Waiting for all jobs to start may not be needed anymore? - _executive->waitForAllJobsToStart(); -} + exec->waitForAllJobsToStart(); -util::HistogramRolling histoBuildAndS("&&&uj histoBuildAndS", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + // we only care about per-chunk info for ASYNC queries + if (_async) { + std::lock_guard lock(chunksMtx); + _qMetaAddChunks(chunks); + } +} void UserQuerySelect::buildAndSendUberJobs() { // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? @@ -328,18 +329,32 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_DEBUG, funcN << " start " << _uberJobMaxChunks); // Ensure `_monitor()` doesn't do anything until everything is ready. - if (!_executive->isReadyToExecute()) { + auto exec = _executive; + if (exec == nullptr) { + LOGS(_log, LOG_LVL_ERROR, funcN << " called with null exec " << getQueryIdString()); + return; + } + + if (!exec->isAllJobsCreated()) { LOGS(_log, LOG_LVL_INFO, funcN << " executive isn't ready to generate UberJobs."); return; } + if (exec->getCancelled() || exec->getSuperfluous()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive cancelled."); + } + + if (exec->getSuperfluous()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive superfluous, result already found."); + } + // Only one thread should be generating UberJobs for this user query at any given time. lock_guard fcLock(_buildUberJobMtx); - LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << _executive->getTotalJobs()); + LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << exec->getTotalJobs()); vector uberJobs; - qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); + qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = exec->unassignedChunksInQuery(); if (unassignedChunksInQuery.empty()) { LOGS(_log, LOG_LVL_DEBUG, funcN << " no unassigned Jobs"); return; @@ -388,15 +403,13 @@ void UserQuerySelect::buildAndSendUberJobs() { map workerJobMap; vector missingChunks; - auto startassign = CLOCK::now(); //&&& // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in // numerical order. The workers run shared scans in numerical order of chunkId numbers. // Numerical order keeps the number of partially complete UberJobs running on a worker to a minimum, // and should minimize the time for the first UberJob on the worker to complete. for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { - bool const increaseAttemptCount = true; - jqPtr->getDescription()->incrAttemptCount(_executive, increaseAttemptCount); + jqPtr->getDescription()->incrAttemptCount(exec, increaseAttemptCount); // If too many workers are down, there will be a chunk that cannot be found. // Just continuing should leave jobs `unassigned` with their attempt count @@ -404,9 +417,8 @@ void UserQuerySelect::buildAndSendUberJobs() { // attempt count will reach max and the query will be cancelled auto lambdaMissingChunk = [&](string const& msg) { missingChunks.push_back(chunkId); - //&&&bool const increaseAttemptCount = true; - //&&&jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); - LOGS(_log, LOG_LVL_ERROR, msg); + auto logLvl = (missingChunks.size() % 1000 == 1) ? LOG_LVL_WARN : LOG_LVL_TRACE; + LOGS(_log, logLvl, msg); }; auto iter = chunkMapPtr->find(chunkId); @@ -460,8 +472,8 @@ void UserQuerySelect::buildAndSendUberJobs() { auto ujId = _uberJobIdSeq++; // keep ujId consistent string uberResultName = _ttn->make(ujId); auto respHandler = make_shared(_infileMerger, uberResultName); - auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), ujId, - _qMetaCzarId, targetWorker); + auto uJob = qdisp::UberJob::create(exec, respHandler, exec->getId(), ujId, _qMetaCzarId, + targetWorker); uJob->setWorkerContactInfo(wInfUJ->wInf); wInfUJ->uberJobPtr = uJob; }; @@ -470,16 +482,12 @@ void UserQuerySelect::buildAndSendUberJobs() { if (wInfUJ->uberJobPtr->getJobCount() >= _uberJobMaxChunks) { // Queue the UberJob to be sent to a worker - _executive->addAndQueueUberJob(wInfUJ->uberJobPtr); + exec->addAndQueueUberJob(wInfUJ->uberJobPtr); - // Clear the pinter so a new UberJob is created later if needed. + // Clear the pointer so a new UberJob is created later if needed. wInfUJ->uberJobPtr = nullptr; } } - auto endassign = CLOCK::now(); //&&& - std::chrono::duration secsassign = endassign - startassign; // &&& - histoBuildAndS.addEntry(endassign, secsassign.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoBuildAndS.getString("")); if (!missingChunks.empty()) { string errStr = funcN + " a worker could not be found for these chunks "; @@ -495,18 +503,23 @@ void UserQuerySelect::buildAndSendUberJobs() { if (winfUjPtr != nullptr) { auto& ujPtr = winfUjPtr->uberJobPtr; if (ujPtr != nullptr) { - _executive->addAndQueueUberJob(ujPtr); + exec->addAndQueueUberJob(ujPtr); } } } - LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); + LOGS(_log, LOG_LVL_DEBUG, funcN << " " << exec->dumpUberJobCounts()); } /// Block until a submit()'ed query completes. /// @return the QueryState indicating success or failure QueryState UserQuerySelect::join() { - bool successful = _executive->join(); // Wait for all data + auto exec = _executive; + if (exec == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "UserQuerySelect::join() called with null exec " << getQueryIdString()); + return ERROR; + } + bool successful = exec->join(); // Wait for all data // Since all data are in, run final SQL commands like GROUP BY. size_t collectedBytes = 0; int64_t finalRows = 0; @@ -518,7 +531,7 @@ QueryState UserQuerySelect::join() { _messageStore->addMessage(-1, "MERGE", 1105, "Failure while merging result", MessageSeverity::MSG_ERROR); } - _executive->updateProxyMessages(); + exec->updateProxyMessages(); try { // The lock is required to prevent the race condition if the query cancellation @@ -533,7 +546,7 @@ QueryState UserQuerySelect::join() { // Update the permanent message table. _qMetaUpdateMessages(); - int64_t collectedRows = _executive->getTotalResultRows(); + int64_t collectedRows = exec->getTotalResultRows(); // finalRows < 0 indicates there was no postprocessing, so collected rows and final rows should be the // same. if (finalRows < 0) finalRows = collectedRows; @@ -557,7 +570,7 @@ QueryState UserQuerySelect::join() { // Notify workers on the query completion/cancellation to ensure // resources are properly cleaned over there as well. - czar::Czar::getCzar()->getActiveWorkerMap()->addToDoneDeleteFiles(_executive->getId()); + czar::Czar::getCzar()->getActiveWorkerMap()->addToDoneDeleteFiles(exec->getId()); return state; } @@ -578,8 +591,14 @@ void UserQuerySelect::discard() { std::lock_guard lock(_killMutex); if (_killed) return; + auto exec = _executive; + if (exec == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "UserQuerySelect::discard called with null exec " << getQueryIdString()); + return; + } + // Make sure resources are released. - if (_executive && _executive->getNumInflight() > 0) { + if (exec->getNumInflight() > 0) { throw UserQueryError(getQueryIdString() + " Executive unfinished, cannot discard"); } @@ -777,8 +796,9 @@ void UserQuerySelect::qMetaRegister(std::string const& resultLocation, std::stri throw UserQueryError(getQueryIdString() + _errorExtra); } - if (_executive != nullptr) { - _executive->setQueryId(_queryId); + auto exec = _executive; + if (exec != nullptr) { + exec->setQueryId(_qMetaQueryId); } else { LOGS(_log, LOG_LVL_WARN, "No Executive, assuming invalid query"); } diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index 68f7be0923..8508fec0a4 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -251,7 +251,7 @@ void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap auto iter = _awMap.find(wcKey); if (iter == _awMap.end()) { auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " AciveWorker created for " << wcKey); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ActiveWorker created for " << wcKey); _awMap[wcKey] = newAW; if (_czarCancelAfterRestart) { newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); diff --git a/src/czar/ActiveWorker.h b/src/czar/ActiveWorker.h index d462f0d0e1..faf4f5671e 100644 --- a/src/czar/ActiveWorker.h +++ b/src/czar/ActiveWorker.h @@ -103,7 +103,9 @@ class ActiveWorker : public std::enable_shared_from_this { /// query information for queries with czarId `czId` and queryId less than /// or equal to `lastQId`. void setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { - _wqsData->setCzarCancelAfterRestart(czId, lastQId); + if (_cancelAfterCzarResetSent.exchange(true) == false) { + _wqsData->setCzarCancelAfterRestart(czId, lastQId); + } } protojson::WorkerContactInfo::Ptr getWInfo() const; @@ -179,6 +181,9 @@ class ActiveWorker : public std::enable_shared_from_this { State _state{QUESTIONABLE}; ///< current state of this worker. mutable std::mutex _aMtx; ///< protects _wInfo, _state, _qIdDoneKeepFiles, _qIdDoneDeleteFiles + + /// Flag to limit sending of czar cancel after reset message. + std::atomic _cancelAfterCzarResetSent{false}; }; /// This class maintains a list of all workers, indicating which are considered active. diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 0124f81856..c56d976f9e 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -156,7 +156,7 @@ Czar::Czar(string const& configFilePath, string const& czarName) _idCounter(), _uqFactory(), _clientToQuery(), - _monitorSleepTime (_czarConfig->getMonitorSleepTimeMilliSec()), + _monitorSleepTime(_czarConfig->getMonitorSleepTimeMilliSec()), _activeWorkerMap(new ActiveWorkerMap(_czarConfig)) { // set id counter to milliseconds since the epoch, mod 1 year. struct timeval tv; @@ -198,6 +198,7 @@ Czar::Czar(string const& configFilePath, string const& czarName) vector vectRunSizes = util::String::parseToVectInt(vectRunSizesStr, ":", 1); string vectMinRunningSizesStr = _czarConfig->getQdispVectMinRunningSizes(); vector vectMinRunningSizes = util::String::parseToVectInt(vectMinRunningSizesStr, ":", 0); + LOGS(_log, LOG_LVL_INFO, " qdisp config qPoolSize=" << qPoolSize << " maxPriority=" << maxPriority << " vectRunSizes=" << vectRunSizesStr << " -> " << util::prettyCharList(vectRunSizes) @@ -206,12 +207,6 @@ Czar::Czar(string const& configFilePath, string const& czarName) _qdispPool = make_shared(qPoolSize, maxPriority, vectRunSizes, vectMinRunningSizes); qdisp::CzarStats::setup(_qdispPool); - int xrootdCBThreadsMax = _czarConfig->getXrootdCBThreadsMax(); - int xrootdCBThreadsInit = _czarConfig->getXrootdCBThreadsInit(); - LOGS(_log, LOG_LVL_INFO, "config xrootdCBThreadsMax=" << xrootdCBThreadsMax); - LOGS(_log, LOG_LVL_INFO, "config xrootdCBThreadsInit=" << xrootdCBThreadsInit); - int const xrootdSpread = _czarConfig->getXrootdSpread(); - LOGS(_log, LOG_LVL_INFO, "config xrootdSpread=" << xrootdSpread); _queryDistributionTestVer = _czarConfig->getQueryDistributionTestVer(); _commandHttpPool = shared_ptr( @@ -401,45 +396,45 @@ void Czar::killQuery(string const& query, string const& clientId) { int threadId; QueryId queryId; if (ccontrol::UserQueryType::isKill(query, threadId)) { - LOGS(_log, LOG_LVL_DEBUG, "thread ID: " << threadId); + LOGS(_log, LOG_LVL_INFO, "KILL thread ID: " << threadId); lock_guard lock(_mutex); // find it in the client map based in client/thread id ClientThreadId ctId(clientId, threadId); auto iter = _clientToQuery.find(ctId); if (iter == _clientToQuery.end()) { - LOGS(_log, LOG_LVL_INFO, "Cannot find client thread id: " << threadId); - throw std::runtime_error("Unknown thread ID: " + query); + LOGS(_log, LOG_LVL_INFO, "KILL Cannot find client thread id: " << threadId); + throw std::runtime_error("KILL Unknown thread ID: " + query); } uq = iter->second.lock(); } else if (ccontrol::UserQueryType::isCancel(query, queryId)) { - LOGS(_log, LOG_LVL_DEBUG, "query ID: " << queryId); + LOGS(_log, LOG_LVL_INFO, "KILL query ID: " << queryId); lock_guard lock(_mutex); // find it in the client map based in client/thread id auto iter = _idToQuery.find(queryId); if (iter == _idToQuery.end()) { - LOGS(_log, LOG_LVL_INFO, "Cannot find query id: " << queryId); - throw std::runtime_error("Unknown or finished query ID: " + query); + LOGS(_log, LOG_LVL_INFO, "KILL Cannot find query id: " << queryId); + throw std::runtime_error("KILL unknown or finished query ID: " + query); } uq = iter->second.lock(); } else { - throw std::runtime_error("Failed to parse query: " + query); + throw std::runtime_error("KILL failed to parse query: " + query); } // assume this cannot fail or throw if (uq) { - LOGS(_log, LOG_LVL_DEBUG, "Killing query: " << uq->getQueryId()); + LOGS(_log, LOG_LVL_INFO, "KILLing query: " << uq->getQueryId()); // query killing can potentially take very long and we do now want to block // proxy from serving other requests so run it in a detached thread thread killThread([uq]() { uq->kill(); - LOGS(_log, LOG_LVL_DEBUG, "Finished killing query: " << uq->getQueryId()); + LOGS(_log, LOG_LVL_INFO, "Finished KILLing query: " << uq->getQueryId()); }); killThread.detach(); } else { - LOGS(_log, LOG_LVL_DEBUG, "Query has expired/finished: " << query); - throw std::runtime_error("Query has already finished: " + query); + LOGS(_log, LOG_LVL_INFO, "KILL query has expired/finished: " << query); + throw std::runtime_error("KILL query has already finished: " + query); } } diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 82d8fd1e8c..c1ebad63eb 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -31,12 +31,11 @@ // Qserv headers #include "qmeta/QMeta.h" +#include "cconfig/CzarConfig.h" #include "czar/Czar.h" #include "czar/CzarRegistry.h" #include "qmeta/Exceptions.h" #include "util/Bug.h" -#include "util/InstanceCount.h" //&&& -#include "util/Histogram.h" //&&& #include "util/TimeUtils.h" using namespace std; @@ -333,7 +332,7 @@ bool CzarFamilyMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarFamilyMap::_read() start"); // If replacing the map, this may take a bit of time, but it's probably // better to wait for new maps if something changed. - std::lock_guard gLock(_familyMapMtx); // &&& check waiting is really needed + std::lock_guard gLock(_familyMapMtx); qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(_lastUpdateTime); if (_lastUpdateTime == qChunkMap.updateTime) { LOGS(_log, LOG_LVL_DEBUG, @@ -344,7 +343,9 @@ bool CzarFamilyMap::_read() { } // Make the new maps. - shared_ptr familyMapPtr = makeNewMaps(qChunkMap); + auto czConfig = cconfig::CzarConfig::instance(); + bool usingChunkSize = czConfig->getFamilyMapUsingChunkSize(); + shared_ptr familyMapPtr = makeNewMaps(qChunkMap, usingChunkSize); verify(familyMapPtr); @@ -356,13 +357,9 @@ bool CzarFamilyMap::_read() { return true; } -util::HistogramRolling histoMakeNewMaps("&&&uj histoMakeNewMaps", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); - std::shared_ptr CzarFamilyMap::makeNewMaps( - qmeta::QMetaChunkMap const& qChunkMap) { + qmeta::QMetaChunkMap const& qChunkMap, bool usingChunkSize) { // Create new maps. - util::InstanceCount ic("CzarFamilyMap::makeNewMaps&&&"); - auto startMakeMaps = CLOCK::now(); //&&& std::shared_ptr newFamilyMap = make_shared(); // Workers -> Databases map @@ -376,7 +373,10 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( for (qmeta::QMetaChunkMap::ChunkInfo const& chunkInfo : chunks) { try { int64_t chunkNum = chunkInfo.chunk; - CzarChunkMap::SizeT sz = chunkInfo.size; + CzarChunkMap::SizeT sz = 1; + if (usingChunkSize) { + sz = chunkInfo.size; + } LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << "workerdId=" << workerId << " db=" << dbName << " table=" << tableName << " chunk=" << chunkNum << " sz=" << sz); @@ -416,10 +416,6 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( } } - auto endMakeMaps = CLOCK::now(); //&&& - std::chrono::duration secsMakeMaps = endMakeMaps - startMakeMaps; // &&& - histoMakeNewMaps.addEntry(endMakeMaps, secsMakeMaps.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoMakeNewMaps.getString("")); return newFamilyMap; } @@ -428,7 +424,7 @@ void CzarFamilyMap::insertIntoMaps(std::shared_ptr const& newFami CzarChunkMap::SizeT sz) { // Get the CzarChunkMap for this family auto familyName = getFamilyNameFromDbName(dbName); - LOGS(_log, LOG_LVL_INFO, + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " familyInsrt{w=" << workerId << " fN=" << familyName << " dbN=" << dbName << " tblN=" << tableName << " chunk=" << chunkIdNum << " sz=" << sz << "}"); auto& nfMap = *newFamilyMap; diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index 28bc023192..8e00ac001d 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -298,10 +298,13 @@ class CzarFamilyMap { /// Make a new FamilyMapType map including ChunkMap and WorkerChunkMap from the data /// in `qChunkMap`. Each family has its own ChunkMap and WorkerChunkMap. + /// @param qChunkMap - data source for the family map + /// @param usingChunkSize - true if the distribution of chunks will depend on the + /// size of the chunks/ /// /// NOTE: This is likely an expensive operation and should probably only /// be called if new workers have been added or chunks have been moved. - std::shared_ptr makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); + std::shared_ptr makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap, bool usingChunkSize); /// Insert the new element described by the parameters into the `newFamilyMap` as appropriate. void insertIntoMaps(std::shared_ptr const& newFamilyMap, std::string const& workerId, diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 266fdbdbe9..9d134aead5 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -98,13 +98,14 @@ json HttpCzarWorkerModule::_workerCzarComIssue() { } json HttpCzarWorkerModule::_handleJobError(string const& func) { - LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobError start"); + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobError start " << body().objJson); + // Metadata-only responses for the file-based protocol should not have any data // Parse and verify the json message and then kill the UberJob. json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // See qdisp::UberJob::runUberJob() for json message construction. &&& + // TODO:UJ see wbase::UberJobData::responseError for message construction string const targetWorkerId = body().required("workerid"); string const czarName = body().required("czar"); qmeta::CzarId const czarId = body().required("czarid"); @@ -145,9 +146,9 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { // Parse and verify the json message and then have the uberjob import the file. json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // &&& TODO:UJ file response - move construction and parsing - // &&& TODO:UJ to a class so it can be added to WorkerCzarComIssue - // See qdisp::UberJob::runUberJob() for json message construction. &&& + // TODO:UJ file response - move construction and parsing + // TODO:UJ to a class so it can be added to WorkerCzarComIssue + // See wbase::UberJobData::responseFileReady string const targetWorkerId = body().required("workerid"); string const czarName = body().required("czar"); qmeta::CzarId const czarId = body().required("czarid"); @@ -163,6 +164,7 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + to_string(queryId) + " czar=" + to_string(czarId)); } + qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); if (uj == nullptr) { throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No UberJob for qid=") + @@ -170,6 +172,9 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { " czar=" + to_string(czarId)); } + uj->setResultFileSize(fileSize); + exec->checkResultFileSize(fileSize); + auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); jsRet = importRes; diff --git a/src/czar/HttpMonitorModule.cc b/src/czar/HttpMonitorModule.cc index fdce70b05d..6a9b37b089 100644 --- a/src/czar/HttpMonitorModule.cc +++ b/src/czar/HttpMonitorModule.cc @@ -53,7 +53,7 @@ HttpMonitorModule::HttpMonitorModule(string const& context, shared_ptrreplicationInstanceId()); enforceCzarName(func); if (subModuleName == "CONFIG") diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index aad9fdfd31..45f42f8318 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -191,13 +191,13 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { auto jsTest1 = nlohmann::json::parse(test1); qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); - auto familyMap = czFamMap.makeNewMaps(qChunkMap1); + auto familyMap = czFamMap.makeNewMaps(qChunkMap1, true); czar::CzarFamilyMap::verify(familyMap); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 1 passed"); auto jsTest2 = nlohmann::json::parse(test2); qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); - auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2); + auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2, true); czar::CzarFamilyMap::verify(familyMap2); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 2 passed"); } diff --git a/src/http/BaseModule.cc b/src/http/BaseModule.cc index 93d15f6aeb..ddf00a107a 100644 --- a/src/http/BaseModule.cc +++ b/src/http/BaseModule.cc @@ -92,13 +92,15 @@ void BaseModule::checkApiVersion(string const& func, unsigned int minVersion, st void BaseModule::enforceInstanceId(string const& func, string const& requiredInstanceId) const { string const instanceId = method() == "GET" ? query().requiredString("instance_id") : body().required("instance_id"); - debug(func, "instance_id: " + instanceId); + trace(func, "instance_id: " + instanceId); if (instanceId != requiredInstanceId) { throw invalid_argument(context() + func + " Qserv instance identifier mismatch. Client sent '" + instanceId + "' instead of '" + requiredInstanceId + "'."); } } +void BaseModule::trace(string const& msg) const { LOGS(_log, LOG_LVL_TRACE, context() << msg); } + void BaseModule::info(string const& msg) const { LOGS(_log, LOG_LVL_INFO, context() << msg); } void BaseModule::debug(string const& msg) const { LOGS(_log, LOG_LVL_DEBUG, context() << msg); } diff --git a/src/http/BaseModule.h b/src/http/BaseModule.h index bef586d22e..686f3aef9b 100644 --- a/src/http/BaseModule.h +++ b/src/http/BaseModule.h @@ -113,6 +113,10 @@ class BaseModule { RequestBodyJSON& body() { return _body; } // Message loggers for the corresponding log levels + // TODO:UJ all of these defeat the purpose of using macros, warn and error are probably fine to keep + // as they should rarely be called. + void trace(std::string const& msg) const; + void trace(std::string const& context, std::string const& msg) const { trace(context + " " + msg); } void info(std::string const& msg) const; void info(std::string const& context, std::string const& msg) const { info(context + " " + msg); } diff --git a/src/protojson/UberJobMsg.cc b/src/protojson/UberJobMsg.cc index 65564cdf48..e95560de09 100644 --- a/src/protojson/UberJobMsg.cc +++ b/src/protojson/UberJobMsg.cc @@ -49,7 +49,7 @@ namespace lsst::qserv::protojson { UberJobMsg::UberJobMsg(unsigned int metaVersion, std::string const& replicationInstanceId, std::string const& replicationAuthKey, CzarContactInfo::Ptr const& czInfo, string const& workerId, QueryId qId, UberJobId ujId, int rowLimit, int maxTableSizeMB, - ScanInfo::Ptr const& scanInfo_, + ScanInfo::Ptr const& scanInfo_, bool scanInteractive_, std::vector> const& jobs) : _metaVersion(metaVersion), _replicationInstanceId(replicationInstanceId), @@ -60,8 +60,9 @@ UberJobMsg::UberJobMsg(unsigned int metaVersion, std::string const& replicationI _ujId(ujId), _rowLimit(rowLimit), _maxTableSizeMB(maxTableSizeMB), - _scanInfo(scanInfo_) { - + _scanInfo(scanInfo_), + _scanInteractive(scanInteractive_), + _idStr("QID=" + to_string(_qId) + "_ujId=" + to_string(_ujId)) { for (auto& jobPtr : jobs) { // This creates the JobMsg objects for all relates jobs and their fragments. auto jobMsg = JobMsg::create(jobPtr, _jobSubQueryTempMap, _jobDbTablesMap); @@ -82,6 +83,7 @@ json UberJobMsg::serializeJson() const { {"dbtables_map", _jobDbTablesMap->serializeJson()}, {"maxtablesizemb", _maxTableSizeMB}, {"scaninfo", _scanInfo->serializeJson()}, + {"scaninteractive", _scanInteractive}, {"jobs", json::array()}}; auto& jsJobs = ujmJson["jobs"]; @@ -89,7 +91,7 @@ json UberJobMsg::serializeJson() const { jsJobs.emplace_back(jbMsg->serializeJson()); } - LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " &&& ujmJson=" << ujmJson); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " ujmJson=" << ujmJson); return ujmJson; } @@ -123,12 +125,14 @@ UberJobMsg::Ptr UberJobMsg::createFromJson(nlohmann::json const& ujmJson) { auto rowLimit = http::RequestBodyJSON::required(ujmJson, "rowlimit"); auto maxTableSizeMB = http::RequestBodyJSON::required(ujmJson, "maxtablesizemb"); auto czInfo = CzarContactInfo::createFromJson(ujmJson["czarinfo"]); + auto scanInteractive_ = http::RequestBodyJSON::required(ujmJson, "scaninteractive"); auto jsUjJobs = http::RequestBodyJSON::required(ujmJson, "jobs"); std::vector> emptyJobs; Ptr ujmPtr = Ptr(new UberJobMsg(metaVersion, replicationInstanceId, replicationAuthKey, czInfo, - workerId, qId, ujId, rowLimit, maxTableSizeMB, scanInfo_, emptyJobs)); + workerId, qId, ujId, rowLimit, maxTableSizeMB, scanInfo_, + scanInteractive_, emptyJobs)); auto const& jsSubQueriesMap = http::RequestBodyJSON::required(ujmJson, "subqueries_map"); ujmPtr->_jobSubQueryTempMap = JobSubQueryTempMap::createFromJson(jsSubQueriesMap); @@ -166,8 +170,6 @@ JobMsg::JobMsg(std::shared_ptr const& jobPtr, _jobId = descr->id(); _attemptCount = descr->getAttemptCount(); _chunkQuerySpecDb = chunkQuerySpec->db; - _scanRating = chunkQuerySpec->scanInfo->scanRating; - _scanInteractive = chunkQuerySpec->scanInteractive; _chunkId = chunkQuerySpec->chunkId; // Add scan tables (TODO:UJ Verify this is the same for all jobs.) @@ -185,14 +187,12 @@ nlohmann::json JobMsg::serializeJson() const { auto jsJobMsg = nlohmann::json({{"jobId", _jobId}, {"attemptCount", _attemptCount}, {"querySpecDb", _chunkQuerySpecDb}, - {"scanPriority", _scanRating}, - {"scanInteractive", _scanInteractive}, {"chunkId", _chunkId}, {"chunkscantables_indexes", nlohmann::json::array()}, {"queryFragments", json::array()}}); // These are indexes into _jobDbTablesMap, which is shared between all JobMsg in this UberJobMsg. - // &&& TODO:UJ queries appear to work even when "chunkscantables_indexes" is wrong + // TODO:UJ "chunkscantables_indexes" may be unused. auto& jsqCstIndexes = jsJobMsg["chunkscantables_indexes"]; for (auto const& index : _chunkScanTableIndexes) { jsqCstIndexes.push_back(index); @@ -207,13 +207,10 @@ nlohmann::json JobMsg::serializeJson() const { } JobMsg::JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap, - JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int scanRating, - bool scanInteractive, int chunkId) + JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int chunkId) : _jobId(jobId), _attemptCount(attemptCount), _chunkQuerySpecDb(chunkQuerySpecDb), - _scanRating(scanRating), - _scanInteractive(scanInteractive), _chunkId(chunkId), _jobSubQueryTempMap(jobSubQueryTempMap), _jobDbTablesMap(jobDbTablesMap) {} @@ -224,14 +221,12 @@ JobMsg::Ptr JobMsg::createFromJson(nlohmann::json const& ujJson, JobId jobId = http::RequestBodyJSON::required(ujJson, "jobId"); int attemptCount = http::RequestBodyJSON::required(ujJson, "attemptCount"); string chunkQuerySpecDb = http::RequestBodyJSON::required(ujJson, "querySpecDb"); - int scanRating = http::RequestBodyJSON::required(ujJson, "scanPriority"); - bool scanInteractive = http::RequestBodyJSON::required(ujJson, "scanInteractive"); int chunkId = http::RequestBodyJSON::required(ujJson, "chunkId"); json jsQFrags = http::RequestBodyJSON::required(ujJson, "queryFragments"); - Ptr jMsgPtr = Ptr(new JobMsg(jobSubQueryTempMap, jobDbTablesMap, jobId, attemptCount, chunkQuerySpecDb, - scanRating, scanInteractive, chunkId)); + Ptr jMsgPtr = Ptr( + new JobMsg(jobSubQueryTempMap, jobDbTablesMap, jobId, attemptCount, chunkQuerySpecDb, chunkId)); json jsChunkTblIndexes = http::RequestBodyJSON::required(ujJson, "chunkscantables_indexes"); jMsgPtr->_chunkScanTableIndexes = jsChunkTblIndexes.get>(); jMsgPtr->_jobFragments = diff --git a/src/protojson/UberJobMsg.h b/src/protojson/UberJobMsg.h index d5f6ade9e0..1962c7feb7 100644 --- a/src/protojson/UberJobMsg.h +++ b/src/protojson/UberJobMsg.h @@ -65,7 +65,7 @@ class JobSubQueryTempMap { static Ptr create() { return Ptr(new JobSubQueryTempMap()); } - /// &&& doc + /// Create JobSubQueryTempMap from result of serializeJson(). static Ptr createFromJson(nlohmann::json const& ujJson); /// Find or insert qTemp into the map and return its index. @@ -87,7 +87,7 @@ class JobSubQueryTempMap { /// The same db+table name pairs recur frequently, so the individual occurrences /// will be replaced with an integer index and use this class to recover the /// complete names. -class JobDbTablesMap { // &&& this class can probably be deleted +class JobDbTablesMap { // TODO:UJ this class can probably be deleted public: using Ptr = std::shared_ptr; @@ -97,7 +97,7 @@ class JobDbTablesMap { // &&& this class can probably be deleted static Ptr create() { return Ptr(new JobDbTablesMap()); } - /// &&& doc + /// Create JobDbTablesMap from result of serializeJson(). static Ptr createFromJson(nlohmann::json const& ujJson); /// Find or insert the db.table pair into the map and return its index. @@ -107,11 +107,11 @@ class JobDbTablesMap { // &&& this class can probably be deleted /// @throws std::out_of_range std::pair getDbTable(int index) { return _dbTableMap.at(index); } - /// &&& TODO:UJ compare with scan rating for entire UberJob + /// TODO:UJ compare with scan rating for entire UberJob void setScanRating(int index, int scanRating, bool lockInMemory); /// Return scanRating(int) and lockInMemory(bool) for the dbTable at `index`. - /// TODO:UJ &&& lockInMemory is expected to go away. + /// TODO:UJ lockInMemory is expected to go away. std::pair getScanRating(int index) { return _scanRatingMap[index]; } nlohmann::json serializeJson() const; @@ -145,7 +145,7 @@ class JobFragment { JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& dbTablesMap); - /// &&& doc + /// Create JobFragment from the serializeJson() result. static VectPtr createVectFromJson(nlohmann::json const& ujJson, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& dbTablesMap); @@ -162,19 +162,19 @@ class JobFragment { private: JobFragment(JobSubQueryTempMap::Ptr const& subQueryTemplates, JobDbTablesMap::Ptr const& dbTablesMap); - /// &&& doc + /// Add the required data for a query fragment. static void _addFragment(std::vector& jFragments, DbTableSet const& subChunkTables, std::vector const& subchunkIds, std::vector const& queries, JobSubQueryTempMap::Ptr const& subQueryTemplates, JobDbTablesMap::Ptr const& dbTablesMap); - JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< &&& doc - std::vector _jobSubQueryTempIndexes; ///< &&& doc + JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< Pointer to indexed list of subquery fragments. + std::vector _jobSubQueryTempIndexes; ///< List of subquery template indexes. - JobDbTablesMap::Ptr _jobDbTablesMap; ///< &&& doc - std::vector _jobDbTablesIndexes; ///< &&& doc + JobDbTablesMap::Ptr _jobDbTablesMap; ///< Pointer to the tables map + std::vector _jobDbTablesIndexes; ///< List of tables used. - std::vector _subchunkIds; ///< &&& doc + std::vector _subchunkIds; ///< List of subchunks for this chunk. }; /// This class is used to store the information for a single Job (the queries and metadata @@ -194,7 +194,7 @@ class JobMsg { JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap); - /// &&& doc + /// Create a Job message from the serializeJson() results. static Ptr createFromJson(nlohmann::json const& ujJson, JobSubQueryTempMap::Ptr const& subQueryTemplates, JobDbTablesMap::Ptr const& dbTablesMap); @@ -204,8 +204,6 @@ class JobMsg { JobId getJobId() const { return _jobId; } int getAttemptCount() const { return _attemptCount; } std::string getChunkQuerySpecDb() const { return _chunkQuerySpecDb; } - int getScanRating() const { return _scanRating; } - bool getScanInteractive() const { return _scanInteractive; } int getChunkId() const { return _chunkId; } std::vector const& getChunkScanTableIndexes() const { return _chunkScanTableIndexes; } @@ -217,20 +215,18 @@ class JobMsg { JobDbTablesMap::Ptr const& jobDbTablesMap); JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap, - JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int scanRating, - bool scanInteractive, int chunkId); + JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int chunkId); JobId _jobId; int _attemptCount; - std::string _chunkQuerySpecDb; - int _scanRating; - bool _scanInteractive; + std::string _chunkQuerySpecDb; // &&& remove, use value for UJ int _chunkId; JobFragment::VectPtr _jobFragments{new JobFragment::Vect()}; JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< Map of all query templates related to this UberJob. JobDbTablesMap::Ptr _jobDbTablesMap; ///< Map of all db.tables related to this UberJob. + // &&& remove, use value for UJ std::vector _chunkScanTableIndexes; ///< list of indexes into _jobDbTablesMap. }; @@ -252,10 +248,10 @@ class UberJobMsg : public std::enable_shared_from_this { static Ptr create(unsigned int metaVersion, std::string const& replicationInstanceId, std::string const& replicationAuthKey, CzarContactInfo::Ptr const& czInfo, WorkerContactInfo::Ptr const& wInfo, QueryId qId, UberJobId ujId, int rowLimit, - int maxTableSizeMB, ScanInfo::Ptr const& scanInfo_, + int maxTableSizeMB, ScanInfo::Ptr const& scanInfo_, bool scanInteractive_, std::vector> const& jobs) { return Ptr(new UberJobMsg(metaVersion, replicationInstanceId, replicationAuthKey, czInfo, wInfo->wId, - qId, ujId, rowLimit, maxTableSizeMB, scanInfo_, jobs)); + qId, ujId, rowLimit, maxTableSizeMB, scanInfo_, scanInteractive_, jobs)); } static Ptr createFromJson(nlohmann::json const& ujJson); @@ -277,11 +273,16 @@ class UberJobMsg : public std::enable_shared_from_this { ScanInfo::Ptr getScanInfo() const { return _scanInfo; } + bool getScanInteractive() const { return _scanInteractive; } + + std::string const& getIdStr() const { return _idStr; } + private: UberJobMsg(unsigned int metaVersion, std::string const& replicationInstanceId, std::string const& replicationAuthKey, CzarContactInfo::Ptr const& czInfo, std::string const& workerId, QueryId qId, UberJobId ujId, int rowLimit, int maxTableSizeMB, - ScanInfo::Ptr const& scanInfo_, std::vector> const& jobs); + ScanInfo::Ptr const& scanInfo_, bool scanInteractive, + std::vector> const& jobs); unsigned int _metaVersion; // "version", http::MetaModule::version // czar @@ -303,7 +304,12 @@ class UberJobMsg : public std::enable_shared_from_this { /// List of all job data in this UberJob. "jobs", json::array() JobMsg::VectPtr _jobMsgVect{new JobMsg::Vect()}; - ScanInfo::Ptr _scanInfo{ScanInfo::create()}; ///< &&& doc + ScanInfo::Ptr _scanInfo{ScanInfo::create()}; ///< Information for shared scan rating. + + /// True if the user query has been designated interactive (quick + high priority) + bool _scanInteractive; + + std::string const _idStr; }; } // namespace lsst::qserv::protojson diff --git a/src/protojson/WorkerQueryStatusData.h b/src/protojson/WorkerQueryStatusData.h index 73aebe2449..eeb93f2847 100644 --- a/src/protojson/WorkerQueryStatusData.h +++ b/src/protojson/WorkerQueryStatusData.h @@ -385,7 +385,7 @@ class WorkerQueryStatusData { /// with that czar. Result files will remain until garbage cleanup or the czar /// calls for their removal. /// TODO:UJ &&& UberJob complete messages that failed to be sent to the czar -/// TODO:UJ &&& will be added to this message. uber job file response +/// TODO:UJ &&& will be added to this message. (uberjob file response) /// Upon successful completion, the worker will clear all values set by the /// the czar. /// Currently, this message is expected to only be needed rarely. diff --git a/src/protojson/testUberJobMsg.cc b/src/protojson/testUberJobMsg.cc index 32412e8658..95f8d8627f 100644 --- a/src/protojson/testUberJobMsg.cc +++ b/src/protojson/testUberJobMsg.cc @@ -46,23 +46,16 @@ namespace test = boost::test_tools; using namespace lsst::qserv::protojson; BOOST_AUTO_TEST_SUITE(Suite) -#if 0 //&&& -std::string testA() { - std::string ta = - R"({"maxtablesizemb":5432,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter` AS `qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; - return ta; -} -#endif // &&& string testA() { string ta = - R"({"maxtablesizemb":5432,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; + R"({"maxtablesizemb":5432,"scaninteractive":true,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; return ta; } string testB() { string tb = - R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":39,"worker":"db04"})"; + R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"scaninteractive":false,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":39,"worker":"db04"})"; return tb; } diff --git a/src/qana/QueryMapping.h b/src/qana/QueryMapping.h index 585971f976..2e8dca319b 100644 --- a/src/qana/QueryMapping.h +++ b/src/qana/QueryMapping.h @@ -92,8 +92,6 @@ class QueryMapping { bool hasParameter(Parameter p) const; DbTableSet const& getSubChunkTables() const { return _subChunkTables; } - std::string dump() const { return std::string("&&& NEED CODE"); } - private: ParameterMap _subs; DbTableSet _subChunkTables; diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index e88cd488ba..b2b6be3469 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -90,18 +90,17 @@ namespace lsst::qserv::qdisp { //////////////////////////////////////////////////////////////////////// // class Executive implementation //////////////////////////////////////////////////////////////////////// -Executive::Executive(ExecutiveConfig const& cfg, shared_ptr const& ms, +Executive::Executive(int secondsBetweenUpdates, shared_ptr const& ms, util::QdispPool::Ptr const& qdispPool, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, shared_ptr const& querySession) - : _config(cfg), - _messageStore(ms), + : _messageStore(ms), _qdispPool(qdispPool), _queryProgress(queryProgress), _queryProgressHistory(queryProgressHistory), + _secondsBetweenQMetaUpdates(chrono::seconds(secondsBetweenUpdates)), _querySession(querySession) { - _secondsBetweenQMetaUpdates = chrono::seconds(_config.secondsBetweenChunkUpdates); _setupLimit(); qdisp::CzarStats::get()->addQuery(); } @@ -125,17 +124,18 @@ Executive::~Executive() { } } } + qdisp::CzarStats::get()->untrackQueryProgress(_id); } -Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptr const& ms, + +Executive::Ptr Executive::create(int secsBetweenUpdates, shared_ptr const& ms, std::shared_ptr const& qdispPool, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, shared_ptr const& querySession, boost::asio::io_service& asioIoService) { LOGS(_log, LOG_LVL_DEBUG, "Executive::" << __func__); - Executive::Ptr ptr( - new Executive(c, ms, sharedResources, queryProgress, queryProgressHistory, querySession)); + Executive::Ptr ptr(new Executive(secsBetweenUpdates, ms, qdispPool, queryProgress, queryProgressHistory, querySession)); // Start the query progress monitoring timer (if enabled). The query status // will be sampled on each expiration event of the timer. Note that the timer @@ -185,6 +185,10 @@ void Executive::setQueryId(QueryId id) { } _id = id; _idStr = QueryIdHelper::makeIdStr(_id); + + // Insert into the global executive map. + czar::Czar::getCzar()->insertExecutive(_id, shared_from_this()); + if (_queryProgressHistory != nullptr) { try { _queryProgressHistory->track(_id); @@ -216,20 +220,22 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getJobId()); { - lock_guard lock(_cancelled.getMutex()); - if (_cancelled) { - LOGS(_log, LOG_LVL_DEBUG, - "Executive already cancelled, ignoring add(" << jobDesc->id() << ")"); - return nullptr; + { + lock_guard lock(_cancelled.getMutex()); + if (_cancelled) { + LOGS(_log, LOG_LVL_DEBUG, + "Executive already cancelled, ignoring add(" << jobDesc->id() << ")"); + return nullptr; + } } - if (!_addJobToMap(jobQuery)) { - LOGS(_log, LOG_LVL_ERROR, "Executive ignoring duplicate job add"); + if (!_track(jobQuery->getJobId(), jobQuery)) { + LOGS(_log, LOG_LVL_ERROR, "Executive ignoring duplicate track add"); return jobQuery; } - if (!_track(jobQuery->getJobId(), jobQuery)) { - LOGS(_log, LOG_LVL_ERROR, "Executive ignoring duplicate track add"); + if (!_addJobToMap(jobQuery)) { + LOGS(_log, LOG_LVL_ERROR, "Executive ignoring duplicate job add"); return jobQuery; } @@ -255,28 +261,12 @@ void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { } } -/* &&& -void Executive::queueUberJob(std::shared_ptr const& uberJob) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&&uj queueUberJob"); - auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; - - auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(runUberJobFunc)); - _jobStartCmdList.push_back(cmd); - if (_scanInteractive) { - _qdispPool->queCmd(cmd, 0); - } else { - _qdispPool->queCmd(cmd, 1); - } -} -*/ - void Executive::addAndQueueUberJob(shared_ptr const& uj) { { lock_guard lck(_uberJobsMapMtx); - UberJobId ujId = uj->getJobId(); + UberJobId ujId = uj->getUjId(); _uberJobsMap[ujId] = uj; - //&&&uj->setAdded(); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); } auto runUberJobFunc = [uj](util::CmdData*) { uj->runUberJob(); }; @@ -447,21 +437,23 @@ void Executive::markCompleted(JobId jobId, bool success) { } _unTrack(jobId); if (!success && !isRowLimitComplete()) { - LOGS(_log, LOG_LVL_ERROR, - "Executive: requesting squash, cause: " << " failed (code=" << err.getCode() << " " + auto logLvl = (_cancelled) ? LOG_LVL_ERROR : LOG_LVL_TRACE; + LOGS(_log, logLvl, + "Executive: requesting cancel, cause: " << " failed (code=" << err.getCode() << " " << err.getMsg() << ")"); - squash(); // ask to squash + squash(string("markComplete error ") + err.getMsg()); // ask to squash } } -void Executive::squash() { +void Executive::squash(string const& note) { bool alreadyCancelled = _cancelled.exchange(true); if (alreadyCancelled) { LOGS(_log, LOG_LVL_DEBUG, "Executive::squash() already cancelled! refusing. qid=" << getId()); return; } - LOGS(_log, LOG_LVL_INFO, "Executive::squash Trying to cancel all queries... qid=" << getId()); + LOGS(_log, LOG_LVL_WARN, + "Executive::squash Trying to cancel all queries... qid=" << getId() << " " << note); deque jobsToCancel; { lock_guard lockJobMap(_jobMapMtx); @@ -470,8 +462,10 @@ void Executive::squash() { } } + int cancelCount = 0; for (auto const& job : jobsToCancel) { job->cancel(); + ++cancelCount; } // Send a message to all workers saying this czarId + queryId is cancelled. @@ -481,12 +475,17 @@ void Executive::squash() { // cancelled. bool const deleteResults = true; sendWorkersEndMsg(deleteResults); - LOGS(_log, LOG_LVL_DEBUG, "Executive::squash done"); + LOGS(_log, LOG_LVL_DEBUG, "Executive::squash done canceled " << cancelCount << " Jobs"); } void Executive::_squashSuperfluous() { if (_cancelled) { - LOGS(_log, LOG_LVL_INFO, "squashSuperfluous() irrelevant as query already cancelled"); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " irrelevant as query already cancelled"); + return; + } + + if (_superfluous.exchange(true) == true) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " irrelevant as query already superfluous"); return; } @@ -505,13 +504,15 @@ void Executive::_squashSuperfluous() { } } + int cancelCount = 0; for (auto const& job : jobsToCancel) { job->cancel(true); + ++cancelCount; } bool const keepResults = false; sendWorkersEndMsg(keepResults); - LOGS(_log, LOG_LVL_DEBUG, "Executive::squashSuperfluous done"); + LOGS(_log, LOG_LVL_DEBUG, "Executive::squashSuperfluous done canceled " << cancelCount << " Jobs"); } void Executive::sendWorkersEndMsg(bool deleteResults) { @@ -705,6 +706,7 @@ void Executive::_waitAllUntilEmpty() { int moreDetailThreshold = 10; int complainCount = 0; const chrono::seconds statePrintDelay(5); + // Loop until all jobs have completed and all jobs have been created. while (!_incompleteJobs.empty()) { count = _incompleteJobs.size(); if (count != lastCount) { @@ -778,6 +780,40 @@ void Executive::checkLimitRowComplete() { _squashSuperfluous(); } +void Executive::checkResultFileSize(uint64_t fileSize) { + _totalResultFileSize += fileSize; + if (_cancelled) return; + + size_t const MB_SIZE_BYTES = 1024 * 1024; + uint64_t maxResultTableSizeBytes = cconfig::CzarConfig::instance()->getMaxTableSizeMB() * MB_SIZE_BYTES; + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << " sz=" << fileSize << " total=" << _totalResultFileSize + << " max=" << maxResultTableSizeBytes); + if (_totalResultFileSize > maxResultTableSizeBytes) { + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " total=" << _totalResultFileSize << " max=" << maxResultTableSizeBytes); + // _totalResultFileSize may include non zero values from dead UberJobs, + // so recalculate it to verify. + uint64_t total = 0; + { + lock_guard lck(_uberJobsMapMtx); + for (auto const& [ujId, ujPtr] : _uberJobsMap) { + total += ujPtr->getResultFileSize(); + } + _totalResultFileSize = total; + } + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << "recheck total=" << total << " max=" << maxResultTableSizeBytes); + if (total > maxResultTableSizeBytes) { + LOGS(_log, LOG_LVL_ERROR, "Executive: requesting squash, result file size too large " << total); + ResponseHandler::Error err(util::ErrorCode::CZAR_RESULT_TOO_LARGE, + string("Incomplete result already too large ") + to_string(total)); + _multiError.push_back(err); + squash("czar, file too large"); + } + } +} + ostream& operator<<(ostream& os, Executive::JobMap::value_type const& v) { auto const& status = v.second->getStatus(); os << v.first << ": " << *status; diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 4a49955a72..bce5c670ec 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -89,17 +89,6 @@ class QdispPool; // This header declarations namespace lsst::qserv::qdisp { -struct ExecutiveConfig { - typedef std::shared_ptr Ptr; - ExecutiveConfig(std::string const& serviceUrl_, int secsBetweenChunkUpdates_) - : serviceUrl(serviceUrl_), secondsBetweenChunkUpdates(secsBetweenChunkUpdates_) {} - ExecutiveConfig(int, int) : serviceUrl(getMockStr()) {} - - std::string serviceUrl; ///< XrdSsi service URL, e.g. localhost:1094 - int secondsBetweenChunkUpdates; ///< Seconds between QMeta chunk updates. - static std::string getMockStr() { return "Mock"; } -}; - /// class Executive manages the execution of jobs for a UserQuery. class Executive : public std::enable_shared_from_this { public: @@ -109,9 +98,7 @@ class Executive : public std::enable_shared_from_this { typedef std::map> ChunkIdJobMapType; /// Construct an Executive. - /// If c->serviceUrl == ExecutiveConfig::getMockStr(), then use XrdSsiServiceMock - /// instead of a real XrdSsiService - static Executive::Ptr create(ExecutiveConfig const& c, std::shared_ptr const& ms, + static Executive::Ptr create(int secsBetweenUpdates, std::shared_ptr const& ms, std::shared_ptr const& qdispPool, std::shared_ptr const& queryProgress, std::shared_ptr const& queryProgressHistory, @@ -155,7 +142,7 @@ class Executive : public std::enable_shared_from_this { void markCompleted(JobId refNum, bool success); /// Squash all the jobs. - void squash(); + void squash(std::string const& note); bool getEmpty() { return _empty; } @@ -179,6 +166,9 @@ class Executive : public std::enable_shared_from_this { /// @return true if cancelled bool getCancelled() { return _cancelled; } + /// Return true if LIMIT conditions met. + bool getSuperfluous() { return _superfluous; } + std::shared_ptr getQdispPool() { return _qdispPool; } /// Add 'rowCount' to the total number of rows in the result table. @@ -218,13 +208,13 @@ class Executive : public std::enable_shared_from_this { // The below value should probably be based on the user query, with longer sleeps for slower queries. int getAttemptSleepSeconds() const { return 15; } // As above or until added to config file. - int getMaxAttempts() const { return 5; } // Should be set by config + int getMaxAttempts() const { return 5; } // TODO:UJ Should be set by config - /// Calling this indicates the executive is ready to create and execute UberJobs. - void setReadyToExecute() { _readyToExecute = true; } + /// Calling this indicates all Jobs for this user query have been created. + void setAllJobsCreated() { _allJobsCreated = true; } - /// Returns true if the executive is ready to create and execute UberJobs. - bool isReadyToExecute() { return _readyToExecute; } + /// Returns true if all jobs have been created. + bool isAllJobsCreated() { return _allJobsCreated; } /// Send a message to all workers to cancel this query. /// @param deleteResults - If true, delete all result files for this query on the workers. @@ -240,9 +230,16 @@ class Executive : public std::enable_shared_from_this { /// Return a pointer to _scanInfo. protojson::ScanInfo::Ptr getScanInfo() { return _scanInfo; } + /// Add fileSize to `_totalResultFileSize` and check if it exceeds limits. + /// If it is too large, check the value against existing UberJob result + /// sizes as `_totalResultFileSize` may include failed UberJobs. + /// If the sum of all UberJob result files size is too large, + /// cancel this user query. + void checkResultFileSize(uint64_t fileSize = 0); + protected: - Executive(ExecutiveConfig const& cfg, std::shared_ptr const& ms, - std::shared_ptr const& qdispPool, + Executive(int secondsBetweenUpdates, std::shared_ptr const& ms, + std::shared_ptr const& sharedResources, std::shared_ptr const& queryProgress, std::shared_ptr const& queryProgressHistory, std::shared_ptr const& querySession); @@ -271,7 +268,6 @@ class Executive : public std::enable_shared_from_this { /// The stats are pushed to qdisp::CzarStats. void _updateStats() const; - ExecutiveConfig _config; ///< Personal copy of config std::atomic _empty{true}; std::shared_ptr _messageStore; ///< MessageStore for logging @@ -291,6 +287,9 @@ class Executive : public std::enable_shared_from_this { std::atomic _requestCount{0}; ///< Count of submitted jobs util::Flag _cancelled{false}; ///< Has execution been cancelled. + /// Set to true when LIMIT conditions have been satisfied. + std::atomic _superfluous{false}; + // Mutexes mutable std::mutex _incompleteJobsMutex; ///< protect incompleteJobs map. @@ -312,7 +311,7 @@ class Executive : public std::enable_shared_from_this { /// Last time Executive updated QMeta, defaults to epoch for clock. std::chrono::system_clock::time_point _lastQMetaUpdate; /// Minimum number of seconds between QMeta chunk updates (set by config) - std::chrono::seconds _secondsBetweenQMetaUpdates{60}; + std::chrono::seconds _secondsBetweenQMetaUpdates; std::mutex _lastQMetaMtx; ///< protects _lastQMetaUpdate. /// true for interactive scans, once set it doesn't change. @@ -350,10 +349,12 @@ class Executive : public std::enable_shared_from_this { /// Weak pointer to the UserQuerySelect object for this query. std::weak_ptr _userQuerySelect; - /// Flag that is set to true when ready to create and run UberJobs. - std::atomic _readyToExecute{false}; + /// Flag that is set to true when all jobs have been created. + std::atomic _allJobsCreated{false}; protojson::ScanInfo::Ptr _scanInfo; ///< Scan rating and tables. + + std::atomic _totalResultFileSize{0}; ///< Total size of all UberJob result files. }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 660e573305..1d205434ec 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -65,25 +65,31 @@ bool JobDescription::incrAttemptCount(std::shared_ptr const& exec, bo if (increase) { ++_attemptCount; } - if (_attemptCount >= MAX_JOB_ATTEMPTS) { - LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); - return false; - } if (exec != nullptr) { int maxAttempts = exec->getMaxAttempts(); - LOGS(_log, LOG_LVL_INFO, "JoQDescription::" << __func__ << " attempts=" << _attemptCount); + if (_attemptCount > 0) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " attempts=" << _attemptCount); + } if (_attemptCount > maxAttempts) { LOGS(_log, LOG_LVL_ERROR, - "JoQDescription::" << __func__ << " attempts(" << _attemptCount << ") > maxAttempts(" - << maxAttempts << ") cancelling"); + cName(__func__) << " attempts(" << _attemptCount << ") > maxAttempts(" << maxAttempts + << ") cancelling"); exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "max attempts reached " + to_string(_attemptCount) + " " + _qIdStr, util::ErrorCode::INTERNAL); - exec->squash(); + exec->squash(string("incrAttemptCount ") + to_string(_attemptCount)); return false; } } + + if (_attemptCount >= MAX_JOB_ATTEMPTS) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " attemptCount greater than max number of retries " << _attemptCount + << " max=" << MAX_JOB_ATTEMPTS); + return false; + } + return true; } diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 75ca4a33b3..7c28a1698e 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -63,14 +63,16 @@ class JobDescription { std::shared_ptr const& respHandler, std::shared_ptr const& chunkQuerySpec, std::string const& chunkResultName, bool mock = false) { - JobDescription::Ptr jd(new JobDescription(czarId, qId, jobId, resource, respHandler, - chunkQuerySpec, chunkResultName, mock)); + JobDescription::Ptr jd(new JobDescription(czarId, qId, jobId, resource, respHandler, chunkQuerySpec, + chunkResultName, mock)); return jd; } JobDescription(JobDescription const&) = delete; JobDescription& operator=(JobDescription const&) = delete; + std::string cName(const char* fnc) { return std::string("JobDescription::") + fnc + " " + _qIdStr; } + JobId id() const { return _jobId; } ResourceUnit const& resource() const { return _resource; } std::shared_ptr respHandler() { return _respHandler; } @@ -103,6 +105,7 @@ class JobDescription { int _attemptCount{-1}; ///< Start at -1 so that first attempt will be 0, see incrAttemptCount(). ResourceUnit _resource; ///< path, e.g. /q/LSST/23125 + // TODO:UJ delete _respHandler, store errors a different way std::shared_ptr _respHandler; // probably MergingHandler std::shared_ptr _chunkQuerySpec; std::string _chunkResultName; diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 71d9f19ecc..21fcbd14e7 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -53,16 +53,13 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } -JobQuery::~JobQuery() { - LOGS(_log, LOG_LVL_DEBUG, "~JobQuery"); - LOGS(_log, LOG_LVL_WARN, "~JobQuery QID=" << _idStr); -} +JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_TRACE, "~JobQuery QID=" << _idStr); } /// Cancel response handling. Return true if this is the first time cancel has been called. bool JobQuery::cancel(bool superfluous) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); - LOGS(_log, LOG_LVL_DEBUG, "JobQuery::cancel()"); if (_cancelled.exchange(true) == false) { + LOGS(_log, LOG_LVL_TRACE, "JobQuery::cancel() " << superfluous); VMUTEX_NOT_HELD(_jqMtx); lock_guard lock(_jqMtx); @@ -83,7 +80,7 @@ bool JobQuery::cancel(bool superfluous) { } return true; } - LOGS(_log, LOG_LVL_TRACE, "cancel, skipping, already cancelled."); + LOGS(_log, LOG_LVL_TRACE, "JobQuery::cancel, skipping, already cancelled."); return false; } @@ -129,8 +126,7 @@ bool JobQuery::unassignFromUberJob(UberJobId ujId) { _uberJobId = -1; auto exec = _executive.lock(); - // Do not increase the count as it should have been increased when the job was started. - _jobDescription->incrAttemptCount(exec, false); + // Do not increase the attempt count as it should have been increased when the job was started. return true; } diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 7c22d7f74d..10e2bd59af 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -60,7 +60,6 @@ class JobQuery { QueryId getQueryId() const { return _qid; } JobId getJobId() const { return _jobDescription->id(); } std::string const& getIdStr() const { return _idStr; } - std::shared_ptr getRespHandler() { return _jobDescription->respHandler(); } JobDescription::Ptr getDescription() { return _jobDescription; } qmeta::JobStatus::Ptr getStatus() { return _jobStatus; } diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 10f535ff13..00ca071965 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -43,7 +43,6 @@ #include "qproc/ChunkQuerySpec.h" #include "util/Bug.h" #include "util/common.h" -#include "util/Histogram.h" //&&& #include "util/QdispPool.h" // LSST headers @@ -77,8 +76,10 @@ UberJob::UberJob(Executive::Ptr const& executive, std::shared_ptrsetUberJobId(getJobId())) { + if (job->setUberJobId(getUjId())) { lock_guard lck(_jobsMtx); _jobs.push_back(job); success = true; @@ -99,16 +100,16 @@ bool UberJob::addJob(JobQuery::Ptr const& job) { return success; } -util::HistogramRolling histoRunUberJob("&&&uj histoRunUberJob", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); -util::HistogramRolling histoUJSerialize("&&&uj histoUJSerialize", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); - -void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelled +void UberJob::runUberJob() { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj start"); // Build the uberjob payload for each job. nlohmann::json uj; unique_lock jobsLock(_jobsMtx); auto exec = _executive.lock(); + if (exec == nullptr || exec->getCancelled()) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive shutdown"); + return; + } // Send the uberjob to the worker auto const method = http::Method::POST; @@ -117,44 +118,27 @@ void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelle vector const headers = {"Content-Type: application/json"}; auto const& czarConfig = cconfig::CzarConfig::instance(); - int maxTableSizeMB = czarConfig->getMaxTableSizeMB(); + uint64_t maxTableSizeMB = czarConfig->getMaxTableSizeMB(); auto czInfo = protojson::CzarContactInfo::create( czarConfig->name(), czarConfig->id(), czarConfig->replicationHttpPort(), util::get_current_host_fqdn(), czar::Czar::czarStartupTime); auto scanInfoPtr = exec->getScanInfo(); + bool scanInteractive = exec->getScanInteractive(); auto uberJobMsg = protojson::UberJobMsg::create( http::MetaModule::version, czarConfig->replicationInstanceId(), czarConfig->replicationAuthKey(), - czInfo, _wContactInfo, _queryId, _uberJobId, _rowLimit, maxTableSizeMB, scanInfoPtr, _jobs); - auto startserialize = CLOCK::now(); //&&& + czInfo, _wContactInfo, _queryId, _uberJobId, _rowLimit, maxTableSizeMB, scanInfoPtr, + scanInteractive, _jobs); + json request = uberJobMsg->serializeJson(); - auto endserialize = CLOCK::now(); //&&& - std::chrono::duration secsserialize = endserialize - startserialize; // &&& - histoUJSerialize.addEntry(endserialize, secsserialize.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoUJSerialize.getString("")); jobsLock.unlock(); // unlock so other _jobsMtx threads can advance while this waits for transmit - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj c"); - /* &&& - { // &&& testing only, delete - auto parsedReq = protojson::UberJobMsg::createFromJson(request); - json jsParsedReq = parsedReq->serializeJson(); - if (request == jsParsedReq) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj YAY!!! "); - } else { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj noYAY request != jsParsedReq"); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj request=" << request); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj jsParsedReq=" << jsParsedReq); - } - } - */ - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << request); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " REQ " << request); string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " czarPost url=" << url << " request=" << request.dump() << " headers=" << headers[0]); - auto startclient = CLOCK::now(); //&&& auto commandHttpPool = czar::Czar::getCzar()->getCommandHttpPool(); http::ClientConfig clientConfig; @@ -167,10 +151,9 @@ void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelle bool transmitSuccess = false; string exceptionWhat; try { - //&&&util::InstanceCount ic{"runUberJob&&&"}; - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj d"); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " sending"); json const response = client.readAsJson(); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj d1"); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " worker recv"); if (0 != response.at("success").get()) { transmitSuccess = true; } else { @@ -180,20 +163,14 @@ void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelle LOGS(_log, LOG_LVL_WARN, requestContext + " ujresponse failed, ex: " + ex.what()); exceptionWhat = ex.what(); } - auto endclient = CLOCK::now(); //&&& - std::chrono::duration secsclient = endclient - startclient; // &&& - histoRunUberJob.addEntry(endclient, secsclient.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoRunUberJob.getString("")); if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure, try to send jobs elsewhere"); _unassignJobs(); // locks _jobsMtx setStatusIfOk(qmeta::JobStatus::RESPONSE_ERROR, cName(__func__) + " not transmitSuccess " + exceptionWhat); - } else { setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx } - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj runuj end"); return; } @@ -206,6 +183,7 @@ void UberJob::prepScrubResults() { } void UberJob::_unassignJobs() { + LOGS(_log, LOG_LVL_INFO, cName(__func__)); lock_guard lck(_jobsMtx); auto exec = _executive.lock(); if (exec == nullptr) { @@ -214,11 +192,11 @@ void UberJob::_unassignJobs() { } for (auto&& job : _jobs) { string jid = job->getIdStr(); - if (!job->unassignFromUberJob(getJobId())) { + if (!job->unassignFromUberJob(getUjId())) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not unassign job=" << jid << " cancelling"); exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "unable to re-assign " + jid, util::ErrorCode::INTERNAL); - exec->squash(); + exec->squash("_unassignJobs failure"); return; } LOGS(_log, LOG_LVL_DEBUG, @@ -287,8 +265,6 @@ void UberJob::callMarkCompleteFunc(bool success) { _jobs.clear(); } -/// Retrieve and process a result file using the file-based protocol -/// Uses a copy of JobQuery::Ptr instead of _jobQuery as a call to cancel() would reset _jobQuery. json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); @@ -313,7 +289,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return _importResultError(false, "rowLimited", "Enough rows already"); } - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileSize=" << fileSize); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " fileSize=" << fileSize); bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); if (!statusSet) { @@ -322,8 +298,8 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ } weak_ptr ujThis = weak_from_this(); - // TODO:UJ lambda may not be the best way to do this, alsocheck synchronization - may need a mutex for - // merging. + + // fileCollectFunc will be put on the queue to run later. string const idStr = _idStr; auto fileCollectFunc = [ujThis, fileUrl, rowCount, idStr](util::CmdData*) { auto ujPtr = ujThis.lock(); @@ -333,26 +309,37 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return; } uint64_t resultRows = 0; - auto [flushSuccess, flushShouldCancel] = + auto [flushSuccess, mergeHappened] = ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); - LOGS(_log, LOG_LVL_DEBUG, ujPtr->cName(__func__) << "::fileCollectFunc"); + LOGS(_log, LOG_LVL_TRACE, + ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushSuccess + << " mergeHappened=" << mergeHappened); if (!flushSuccess) { - // This would probably indicate malformed file+rowCount or - // writing the result table failed. + bool flushShouldCancel = false; + if (mergeHappened) { + // This would probably indicate malformed file+rowCount or writing the result table failed. + // If any merging happened, the result table is ruined. + LOGS(_log, LOG_LVL_ERROR, + ujPtr->cName(__func__) + << "::fileCollectFunc flushHttp failed after merging, results ruined."); + flushShouldCancel = true; + } else { + // Perhaps something went wrong with file collection, so it is worth trying the jobs again + // by abandoning this UberJob. + LOGS(_log, LOG_LVL_ERROR, + ujPtr->cName(__func__) << "::fileCollectFunc flushHttp failed, retrying Jobs."); + } ujPtr->_importResultError(flushShouldCancel, "mergeError", "merging failed"); } - // At this point all data for this job have been read, there's no point in - // having XrdSsi wait for anything. + // At this point all data for this job have been read. ujPtr->_importResultFinish(resultRows); }; auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); exec->queueFileCollect(cmd); - // If the query meets the limit row complete complete criteria, it will start - // squashing superfluous results so the answer can be returned quickly. - + // The file collection has been queued for later, let the worker know that it's okay so far. json jsRet = {{"success", 1}, {"errortype", ""}, {"note", "queued for collection"}}; return jsRet; } @@ -385,7 +372,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { // TODO:UJ see if recoverable errors can be detected on the workers, or // maybe allow a single retry before sending the error back to the user? bool recoverableError = false; - recoverableError = true; // TODO:UJ delete after testing + if (recoverableError) { // The czar should have new maps before the the new UberJob(s) for // these Jobs are created. (see Czar::_monitor) @@ -395,7 +382,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { int errState = util::ErrorCode::MYSQLEXEC; getRespHandler()->flushHttpError(errorCode, errorMsg, errState); exec->addMultiError(errorCode, errorMsg, errState); - exec->squash(); + exec->squash(string("UberJob::workerError ") + errorMsg); } string errType = to_string(errorCode) + ":" + errorMsg; @@ -414,7 +401,7 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str if (shouldCancel) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failing jobs"); callMarkCompleteFunc(false); // all jobs failed, no retry - exec->squash(); + exec->squash(string("_importResultError shouldCancel")); } else { /// - each JobQuery in _jobs needs to be flagged as needing to be /// put in an UberJob and it's attempt count increased and checked @@ -451,7 +438,7 @@ void UberJob::_importResultFinish(uint64_t resultRows) { if (!statusSet) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to set status, squashing " << getIdStr()); // Something has gone very wrong - exec->squash(); + exec->squash("UberJob::_importResultFinish couldn't set status"); return; } diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index ce719d50d8..6333b3eb23 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -71,9 +71,7 @@ class UberJob : public std::enable_shared_from_this { void killUberJob(); QueryId getQueryId() const { return _queryId; } - UberJobId getJobId() const { - return _uberJobId; - } // &&& TODO:UJ change name when JobBase no longer needed. + UberJobId getUjId() const { return _uberJobId; } std::string const& getIdStr() const { return _idStr; } std::shared_ptr getRespHandler() { return _respHandler; } std::shared_ptr getStatus() { return _jobStatus; } @@ -107,12 +105,17 @@ class UberJob : public std::enable_shared_from_this { /// Get the data for the worker that should handle this UberJob. czar::CzarChunkMap::WorkerChunksData::Ptr getWorkerData() { return _workerData; } - /// Collect and merge the results from the worker. + /// Queue the lambda function to collect and merge the results from the worker. + /// @return a json message indicating success unless the query has been + /// cancelled, limit row complete, or similar. nlohmann::json importResultFile(std::string const& fileUrl, uint64_t rowCount, uint64_t fileSize); /// Handle an error from the worker. nlohmann::json workerError(int errorCode, std::string const& errorMsg); + void setResultFileSize(uint64_t fileSize) { _resultFileSize = fileSize; } + uint64_t getResultFileSize() { return _resultFileSize; } + std::ostream& dumpOS(std::ostream& os) const; std::string dump() const; friend std::ostream& operator<<(std::ostream& os, UberJob const& uj); @@ -160,6 +163,7 @@ class UberJob : public std::enable_shared_from_this { UberJobId const _uberJobId; qmeta::CzarId const _czarId; int const _rowLimit; + uint64_t _resultFileSize = 0; std::string const _idStr; @@ -167,7 +171,7 @@ class UberJob : public std::enable_shared_from_this { czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // TODO:UJ this may not be needed // Contact information for the target worker. - protojson::WorkerContactInfo::Ptr _wContactInfo; // Change to ActiveWorker &&& ??? + protojson::WorkerContactInfo::Ptr _wContactInfo; // TODO:UJ Maybe change to ActiveWorker? }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 34183d0bd1..0a30aabb72 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -145,10 +145,10 @@ class ExecutiveUT : public Executive { ~ExecutiveUT() override = default; - ExecutiveUT(ExecutiveConfig const& cfg, shared_ptr const& ms, + ExecutiveUT(int qmetaTimeBetweenUpdates, shared_ptr const& ms, util::QdispPool::Ptr const& qdispPool, shared_ptr const& qStatus, shared_ptr const& querySession, TestInfo::Ptr const& testInfo_) - : Executive(cfg, ms, qdispPool, qStatus, querySession), testInfo(testInfo_) {} + : Executive(qmetaTimeBetweenUpdates, ms, qdispPool, qStatus, querySession), testInfo(testInfo_) {} void assignJobsToUberJobs() override { vector ujVect; @@ -187,8 +187,8 @@ qdisp::JobDescription::Ptr makeMockJobDescription(qdisp::Executive::Ptr const& e auto cqs = std::make_shared(); // dummy, unused in this case. std::string chunkResultName = "dummyResultTableName"; qmeta::CzarId const czarId = 1; - auto job = qdisp::JobDescription::create(czarId, ex->getId(), sequence, ru, mHandler, - cqs, chunkResultName, true); + auto job = qdisp::JobDescription::create(czarId, ex->getId(), sequence, ru, mHandler, cqs, + chunkResultName, true); return job; } @@ -209,6 +209,7 @@ std::shared_ptr addMockRequests(qdisp::Executive::Ptr const& ex qdisp::JobDescription::Ptr job = makeMockJobDescription(ex, sequence.incr(), ru, msg, rv[j]); jobQuery = ex->add(job); } + ex->setAllJobsCreated(); return jobQuery; } @@ -257,8 +258,6 @@ void timeoutFunc(std::atomic& flagDone, int millisecs) { class SetupTest { public: std::string qrMsg; - std::string str; - qdisp::ExecutiveConfig::Ptr conf; std::shared_ptr ms; util::QdispPool::Ptr qdispPool; qdisp::ExecutiveUT::PtrUT ex; @@ -268,15 +267,12 @@ class SetupTest { SetupTest(const char* request, util::QdispPool::Ptr const& qPool_) : qdispPool(qPool_) { LOGS(_log, LOG_LVL_INFO, "SetupTest start"); qrMsg = request; - str = qdisp::ExecutiveConfig::getMockStr(); - conf = std::make_shared(str, 0); // No updating of QMeta. ms = std::make_shared(); auto tInfo = qdisp::TestInfo::Ptr(new qdisp::TestInfo()); - std::shared_ptr queryProgress; // No updating QProgress, nullptr + std::shared_ptr qProgress; // No updating QProgress, nullptr std::shared_ptr queryProgressHistory; // No updating QProgressHistory, nullptr - ex = qdisp::ExecutiveUT::PtrUT( - new qdisp::ExecutiveUT(*conf, ms, qdispPool, qProgress, + ex = qdisp::ExecutiveUT::PtrUT(new qdisp::ExecutiveUT(60, ms, qdispPool, qProgress, queryProgressHistory, nullptr, testInfo)); LOGS(_log, LOG_LVL_INFO, "SetupTest end"); } @@ -385,7 +381,7 @@ BOOST_AUTO_TEST_CASE(ExecutiveCancel) { // squash SequentialInt sequence(0); tEnv.jqTest = executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 1); - tEnv.ex->squash(); + tEnv.ex->squash("test"); usleep(250000); // Give mock threads a quarter second to complete. tEnv.ex->join(); BOOST_CHECK(tEnv.jqTest->isQueryCancelled() == true); @@ -398,9 +394,9 @@ BOOST_AUTO_TEST_CASE(ExecutiveCancel) { // squash SequentialInt sequence(0); executiveTest(tEnv.ex, sequence, chunkId, tEnv.qrMsg, 20); - tEnv.ex->squash(); - tEnv.ex->squash(); // check that squashing twice doesn't cause issues. - usleep(250000); // Give mock threads a quarter second to complete. + tEnv.ex->squash("test"); + tEnv.ex->squash("test"); // check that squashing twice doesn't cause issues. + usleep(250000); // Give mock threads a quarter second to complete. tEnv.ex->join(); } } diff --git a/src/qhttp/Server.cc b/src/qhttp/Server.cc index 5116255cb6..24801bc350 100644 --- a/src/qhttp/Server.cc +++ b/src/qhttp/Server.cc @@ -57,6 +57,8 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qhttp"); } +using namespace std; + namespace lsst::qserv::qhttp { Server::Ptr Server::create(asio::io_service& io_service, unsigned short port, int backlog, @@ -115,11 +117,11 @@ void Server::_accept() { [](auto& weakSocket) { return weakSocket.expired(); }); auto numExpired = _activeSockets.end() - removed; if (numExpired != 0) { - LOGLS_DEBUG(_log, logger(this) << "purging tracking for " << numExpired << " expired socket(s)"); + LOGLS_TRACE(_log, logger(this) << "purging tracking for " << numExpired << " expired socket(s)"); _activeSockets.erase(removed, _activeSockets.end()); } _activeSockets.push_back(socket); - LOGLS_DEBUG(_log, logger(this) << "tracking new socket"); + LOGLS_TRACE(_log, logger(this) << "tracking new socket"); } auto self = shared_from_this(); @@ -128,13 +130,18 @@ void Server::_accept() { LOGLS_DEBUG(_log, logger(self) << "accept chain exiting"); return; } - if (!ec) { - LOGLS_INFO(_log, logger(self) << logger(socket) << "connect from " << socket->remote_endpoint()); - boost::system::error_code ignore; - socket->set_option(ip::tcp::no_delay(true), ignore); - self->_readRequest(socket); - } else { - LOGLS_ERROR(_log, logger(self) << "accept failed: " << ec.message()); + try { + if (!ec) { + LOGLS_INFO(_log, logger(self) + << logger(socket) << "connect from " << socket->remote_endpoint()); + boost::system::error_code ignore; + socket->set_option(ip::tcp::no_delay(true), ignore); + self->_readRequest(socket); + } else { + LOGLS_ERROR(_log, logger(self) << "accept failed: " << ec.message()); + } + } catch (boost::system::system_error const& bEx) { + LOGS(_log, LOG_LVL_ERROR, "qhttp::Server::_accept lambda threw " << bEx.what()); } self->_accept(); // start accept again for the next incoming connection }); @@ -187,7 +194,7 @@ std::shared_ptr Server::_startTimer(std::shared_ptrlowest_layer().shutdown(ip::tcp::socket::shutdown_both, ignore); socket->lowest_layer().close(ignore); } else if (ec == asio::error::operation_aborted) { - LOGLS_DEBUG(_log, logger(self) << logger(socket) << "read timeout timer canceled"); + LOGLS_TRACE(_log, logger(self) << logger(socket) << "read timeout timer canceled"); } else { LOGLS_ERROR(_log, logger(self) << logger(socket) << "read timeout timer: " << ec.message()); } @@ -209,13 +216,15 @@ void Server::_readRequest(std::shared_ptr socket) { self, socket, [self, socket, startTime, reuseSocket](boost::system::error_code const& ec, std::size_t sent) { chrono::duration elapsed = chrono::steady_clock::now() - startTime; - LOGLS_INFO(_log, logger(self) - << logger(socket) << "request duration " << elapsed.count() << "ms"); + string logStr; + if (LOG_CHECK_LVL(_log, LOG_LVL_INFO)) { + logStr = string("request duration ") + to_string(elapsed.count()) + "ms"; + } if (!ec && *reuseSocket) { - LOGLS_DEBUG(_log, logger(self) << logger(socket) << "lingering"); + LOGLS_INFO(_log, logger(self) << logger(socket) << logStr << " lingering"); self->_readRequest(socket); } else { - LOGLS_DEBUG(_log, logger(self) << logger(socket) << "closing"); + LOGLS_INFO(_log, logger(self) << logger(socket) << logStr << " closing"); boost::system::error_code ignore; socket->lowest_layer().shutdown(ip::tcp::socket::shutdown_both, ignore); socket->lowest_layer().close(ignore); diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index c3bab4dc62..e89c702c63 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -52,6 +52,7 @@ namespace { // Current version of QMeta schema char const VERSION_STR[] = "17"; + LOG_LOGGER _log = LOG_GET("lsst.qserv.qmeta.QMetaMysql"); using lsst::qserv::qmeta::QInfo; diff --git a/src/qproc/ChunkQuerySpec.h b/src/qproc/ChunkQuerySpec.h index 41582368f0..ef0d8325c4 100644 --- a/src/qproc/ChunkQuerySpec.h +++ b/src/qproc/ChunkQuerySpec.h @@ -52,7 +52,6 @@ class ChunkQuerySpec { using Ptr = std::shared_ptr; ChunkQuerySpec() {} - //&&&ChunkQuerySpec(std::string const& db_, int chunkId_, protojson::ScanInfo const& scanInfo_, ChunkQuerySpec(std::string const& db_, int chunkId_, protojson::ScanInfo::Ptr const& scanInfo_, bool scanInteractive_) : db(db_), chunkId(chunkId_), scanInfo(scanInfo_), scanInteractive(scanInteractive_) {} @@ -60,15 +59,13 @@ class ChunkQuerySpec { // Contents could change std::string db{""}; ///< dominant db int chunkId{0}; - //&&&protojson::ScanInfo scanInfo; ///< shared-scan candidates protojson::ScanInfo::Ptr scanInfo; ///< shared-scan candidates // Consider saving subChunkTable templates, and substituting the chunkIds // and subChunkIds into them on-the-fly. bool scanInteractive{false}; DbTableSet subChunkTables; std::vector subChunkIds; - std::vector queries; // &&& remove if possible - std::vector queryTemplates; + std::vector queries; // Consider promoting the concept of container of ChunkQuerySpec // in the hopes of increased code cleanliness. std::shared_ptr nextFragment; ///< ad-hoc linked list (consider removal) diff --git a/src/qproc/QuerySession.cc b/src/qproc/QuerySession.cc index 9bd643265b..969409a4dc 100644 --- a/src/qproc/QuerySession.cc +++ b/src/qproc/QuerySession.cc @@ -391,7 +391,6 @@ std::vector QuerySession::_buildChunkQueries(query::QueryTemplate:: } for (auto&& queryTemplate : queryTemplates) { - LOGS(_log, LOG_LVL_WARN, "&&&uj QuerySession::_buildChunkQueries qt=" << queryTemplate.dump()); std::string str = _context->queryMapping->apply(chunkSpec, queryTemplate); chunkQueries.push_back(std::move(str)); } diff --git a/src/replica/contr/HttpQservMonitorModule.cc b/src/replica/contr/HttpQservMonitorModule.cc index 9e0fd4ba28..ba0f72e348 100644 --- a/src/replica/contr/HttpQservMonitorModule.cc +++ b/src/replica/contr/HttpQservMonitorModule.cc @@ -413,10 +413,10 @@ wbase::TaskSelector HttpQservMonitorModule::_translateTaskSelector(string const& } } selector.maxTasks = query().optionalUInt("max_tasks", 0); - debug(func, "include_tasks=" + replica::bool2str(selector.includeTasks)); - debug(func, "query_ids=" + util::String::toString(selector.queryIds)); - debug(func, "task_states=" + util::String::toString(selector.taskStates)); - debug(func, "max_tasks=" + to_string(selector.maxTasks)); + trace(func, "include_tasks=" + replica::bool2str(selector.includeTasks) + + " query_ids=" + util::String::toString(selector.queryIds) + + " task_states=" + util::String::toString(selector.taskStates) + + " max_tasks=" + to_string(selector.maxTasks)); return selector; } diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 9eff6d90e8..bb21066f70 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -164,92 +164,8 @@ void InfileMerger::mergeCompleteFor(int jobId) { _totalResultSize += _perJobResultSize[jobId]; // TODO:UJ this can probably be simplified } -bool InfileMerger::merge(proto::ResponseSummary const& resp, - std::shared_ptr const& csvStream) { - int const jobId = resp.jobid(); - std::string queryIdJobStr = QueryIdHelper::makeIdStr(resp.queryid(), jobId); - if (!_queryIdStrSet) { - _setQueryIdStr(QueryIdHelper::makeIdStr(resp.queryid())); - } - - // Nothing to do if size is zero. - if (responseData.row_size() == 0) { - return true; - } - - // Do nothing if the query got cancelled for any reason. - if (jq->isQueryCancelled()) { - return true; - } - auto executive = jq->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { - return true; - } - - std::unique_ptr semaLock; - if (_dbEngine != MYISAM) { - // needed for parallel merging with INNODB and MEMORY - semaLock.reset(new util::SemaLock(*_semaMgrConn)); - } - - TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, - bool success) { - if (!success) return; - if (std::chrono::duration const seconds = end - start; seconds.count() > 0) { - qdisp::CzarStats::get()->addXRootDSSIRecvRate(bytes / seconds.count()); - } - }; - auto tct = make_shared>(cbf); - - bool ret = false; - // Add columns to rows in virtFile. - util::Timer virtFileT; - virtFileT.start(); - auto const csvBuffer = mysql::newCsvStreamBuffer(csvStream); - std::string const virtFile = _infileMgr.prepareSrc(csvBuffer); - std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); - virtFileT.stop(); - - size_t tResultSize; - { - std::lock_guard resultSzLock(_mtxResultSizeMtx); - _perJobResultSize[jobId] += resp.transmitsize(); - tResultSize = _totalResultSize + _perJobResultSize[jobId]; - } - if (tResultSize > _maxResultTableSizeBytes) { - std::ostringstream os; - os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable - << " is too large at " << tResultSize << " bytes, max allowed size is " << _maxResultTableSizeBytes - << " bytes"; - LOGS(_log, LOG_LVL_ERROR, os.str()); - _error = util::Error(-1, os.str(), -1); - _resultSizeLimitExceeded.store(true); - return false; - } - - tct->addToValue(resp.transmitsize()); - tct->setSuccess(); - tct.reset(); // stop transmit recieve timer before merging happens. - - // Stop here (if requested) after collecting stats on the amount of data collected - // from workers. - if (_config.debugNoMerge) return true; - - auto start = std::chrono::system_clock::now(); - ret = _applyMysqlMyIsam(infileStatement, resp.transmitsize()); - auto end = std::chrono::system_clock::now(); - auto mergeDur = std::chrono::duration_cast(end - start); - LOGS(_log, LOG_LVL_DEBUG, "mergeDur=" << mergeDur.count()); - if (not ret) { - LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); - } - LOGS(_log, LOG_LVL_DEBUG, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); - - return ret; -} - bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { - UberJobId const uJobId = uberJob->getJobId(); + UberJobId const uJobId = uberJob->getUjId(); std::string queryIdJobStr = uberJob->getIdStr(); if (!_queryIdStrSet) { _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); @@ -289,7 +205,7 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response util::Timer virtFileT; virtFileT.start(); // UberJobs only get one attempt - int resultJobId = makeJobIdAttempt(uberJob->getJobId(), 0); + int resultJobId = makeJobIdAttempt(uberJob->getUjId(), 0); ProtoRowBuffer::Ptr pRowBuffer = std::make_shared( responseData, resultJobId, _jobIdColName, _jobIdSqlType, _jobIdMysqlType); std::string const virtFile = _infileMgr.prepareSrc(pRowBuffer); @@ -332,29 +248,31 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response return true; } - auto start = std::chrono::system_clock::now(); + auto start = CLOCK::now(); switch (_dbEngine) { case MYISAM: ret = _applyMysqlMyIsam(infileStatement, resultSize); break; - case INNODB: // Fallthrough + case INNODB: + [[fallthrough]]; case MEMORY: ret = _applyMysqlInnoDb(infileStatement, resultSize); break; default: throw std::invalid_argument("InfileMerger::_dbEngine is unknown =" + engineToStr(_dbEngine)); } - auto end = std::chrono::system_clock::now(); + auto end = CLOCK::now(); auto mergeDur = std::chrono::duration_cast(end - start); - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "mergeDur=" << mergeDur.count() << " sema(total=" << _semaMgrConn->getTotalCount() << " used=" << _semaMgrConn->getUsedCount() << ")"); + if (not ret) { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); } _invalidJobAttemptMgr.decrConcurrentMergeCount(); - LOGS(_log, LOG_LVL_DEBUG, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); + LOGS(_log, LOG_LVL_TRACE, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); return ret; } diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index 6d2d98fef2..77eb9f796d 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -105,10 +105,6 @@ class InfileMerger { InfileMerger& operator=(InfileMerger const&) = delete; ~InfileMerger() = default; - /// Merge a worker response, which contains a single message - /// @return true if merge was successfully imported. - bool merge(proto::ResponseSummary const& resp, std::shared_ptr const& csvStream); - /// Merge the result data collected over Http. bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); diff --git a/src/util/Error.cc b/src/util/Error.cc index 9f6e39e3f5..79453db26b 100644 --- a/src/util/Error.cc +++ b/src/util/Error.cc @@ -38,10 +38,12 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.util.Error"); namespace lsst::qserv::util { -Error::Error(int code, std::string const& msg, int status) : _code(code), _msg(msg), _status(status) { +Error::Error(int code, std::string const& msg, int status, bool logLvlErr) + : _code(code), _msg(msg), _status(status) { if (_code != ErrorCode::NONE || _msg != "" || _status != ErrorCode::NONE) { // Flushing output as it is likely that this exception will not be caught. - LOGS(_log, LOG_LVL_ERROR, "Error " << *this << std::endl); + int logLvl = (logLvlErr) ? LOG_LVL_ERROR : LOG_LVL_TRACE; + LOGS(_log, logLvl, "Error " << *this << std::endl); } } diff --git a/src/util/Error.h b/src/util/Error.h index c95ec76b06..ad11db124f 100644 --- a/src/util/Error.h +++ b/src/util/Error.h @@ -61,6 +61,7 @@ struct ErrorCode { MYSQLCONNECT, MYSQLEXEC, INTERNAL, + CZAR_RESULT_TOO_LARGE, // Worker errors: WORKER_RESULT_TOO_LARGE }; @@ -73,7 +74,8 @@ struct ErrorCode { */ class Error { public: - Error(int code = ErrorCode::NONE, std::string const& msg = "", int status = ErrorCode::NONE); + Error(int code = ErrorCode::NONE, std::string const& msg = "", int status = ErrorCode::NONE, + bool logLvLErr = true); /** Overload output operator for current class * diff --git a/src/util/InstanceCount.cc b/src/util/InstanceCount.cc index 895698d63b..9960137609 100644 --- a/src/util/InstanceCount.cc +++ b/src/util/InstanceCount.cc @@ -8,6 +8,8 @@ // LSST headers #include "lsst/log/Log.h" +using namespace std; + namespace { // File-scope helpers LOG_LOGGER _log = LOG_GET("lsst.qserv.util.InstanceCount"); @@ -31,17 +33,21 @@ void InstanceCount::_increment(std::string const& source) { auto ret = _instances.insert(entry); auto iter = ret.first; iter->second += 1; - LOGS(_log, LOG_LVL_WARN, - "InstanceCount " << source << " " << iter->first << "=" << iter->second); //&&&DEBUG + LOGS(_log, LOG_LVL_TRACE, "InstanceCount " << source << " " << iter->first << "=" << iter->second); } +uint16_t instanceDestructLogLimiter = 0; + InstanceCount::~InstanceCount() { std::lock_guard lg(_mx); + ++instanceDestructLogLimiter; auto iter = _instances.find(_className); if (iter != _instances.end()) { iter->second -= 1; - LOGS(_log, LOG_LVL_WARN, - "~InstanceCount " << iter->first << "=" << iter->second << " : " << *this); //&&&DEBUG + LOGS(_log, LOG_LVL_TRACE, "~InstanceCount " << iter->first << "=" << iter->second << " : " << *this); + if (instanceDestructLogLimiter % 1000 == 0) { + LOGS(_log, LOG_LVL_DEBUG, "~InstanceCount brief " << *this); + } if (iter->second == 0) { _instances.erase(_className); } diff --git a/src/util/Mutex.h b/src/util/Mutex.h index 991db6b182..62b84e214a 100644 --- a/src/util/Mutex.h +++ b/src/util/Mutex.h @@ -32,7 +32,7 @@ #include "util/Bug.h" -#define USING_VMUTEX 0 // &&& Should be replaced by variable in build. +#define USING_VMUTEX 0 // TODO:UJ Should be replaced by variable in build. #ifdef MUTEX_UNITTEST #define USING_VMUTEX 1 diff --git a/src/util/QdispPool.cc b/src/util/QdispPool.cc index 02d2e1c41f..131200dc98 100644 --- a/src/util/QdispPool.cc +++ b/src/util/QdispPool.cc @@ -99,7 +99,8 @@ util::Command::Ptr PriorityQueue::getCmd(bool wait) { ++localLogLimiter; // Log this every once in while to INFO so there's some idea of system // load without generating crushing amounts of log messages. - if (localLogLimiter % 500 == 0) { + unsigned int const limitMod = 100; + if (localLogLimiter % limitMod == 0) { LOGS(_log, LOG_LVL_INFO, "priQueGet " << _statsStr()); } else { LOGS(_log, LOG_LVL_DEBUG, "priQueGet " << _statsStr()); diff --git a/src/util/ResultFileNameParser.cc b/src/util/ResultFileNameParser.cc new file mode 100644 index 0000000000..eaf4c1ce02 --- /dev/null +++ b/src/util/ResultFileNameParser.cc @@ -0,0 +1,79 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "util/ResultFileNameParser.h" + +// Third party headers +#include "boost/filesystem.hpp" + +// Qserv headers +#include "util/String.h" + +using namespace std; +using namespace nlohmann; +namespace fs = boost::filesystem; + +namespace lsst::qserv::util { + +string const ResultFileNameParser::fileExt = ".proto"; + +ResultFileNameParser::ResultFileNameParser(fs::path const& filePath) : _fileName(filePath.stem().string()) { + _parse(); +} + +ResultFileNameParser::ResultFileNameParser(string const& filePath) + : _fileName(fs::path(filePath).stem().string()) { + _parse(); +} + +json ResultFileNameParser::toJson() const { + return json::object( + {{"czar_id", czarId}, {"query_id", queryId}, {"job_id", jobId}, {"chunk_id", chunkId}}); +} + +bool ResultFileNameParser::operator==(ResultFileNameParser const& rhs) const { + return (czarId == rhs.czarId) && (queryId == rhs.queryId) && (jobId == rhs.jobId) && + (chunkId == rhs.chunkId); +} + +ostream& operator<<(ostream& os, ResultFileNameParser const& parser) { + os << parser.toJson(); + return os; +} + +string ResultFileNameParser::_context(string const& func) { + return "FileChannelShared::ResultFileNameParser::" + func; +} + +void ResultFileNameParser::_parse() { + _taskAttributes = String::parseToVectUInt64(_fileName, "-"); + if (_taskAttributes.size() != 4) { + throw invalid_argument(_context(__func__) + " not a valid result file name: " + _fileName); + } + size_t attrIndex = 0; + _validateAndStoreAttr(attrIndex++, "czarId", czarId); + _validateAndStoreAttr(attrIndex++, "queryId", queryId); + _validateAndStoreAttr(attrIndex++, "jobId", jobId); + _validateAndStoreAttr(attrIndex++, "chunkId", chunkId); +} + +} // namespace lsst::qserv::util diff --git a/src/util/ThreadPool.h b/src/util/ThreadPool.h index 80bc275d17..c25bae28d6 100644 --- a/src/util/ThreadPool.h +++ b/src/util/ThreadPool.h @@ -111,8 +111,11 @@ class CommandForThreadPool : public CommandTracked { /// Note: It is possible for threads to leave the pool and be replaced using leavePool() /// This is usually done when a thread no longer requires significant CPU but has /// to wait for something to happen, like transferring data. -/// _poolThreadCount is a total of all threads in the pool and all threads that have -/// left the pool and this total should not exceed _maxThreadCount. +/// _poolThreadCount is a total of all threads that are still running both in the pool +/// and those that have left the pool but have not finished. +/// _poolThreadCount total should not exceed _maxThreadCount. +/// _maxThreadCount can be significantly larger than the _targetThrdCount. +/// class ThreadPool : public std::enable_shared_from_this { public: using Ptr = std::shared_ptr; diff --git a/src/util/common.cc b/src/util/common.cc index 8c7969c1e3..8f0d0ec5a8 100644 --- a/src/util/common.cc +++ b/src/util/common.cc @@ -29,12 +29,20 @@ #include #include #include +#include // Third-party headers #include "boost/asio.hpp" +// LSST headers +#include "lsst/log/Log.h" + using namespace std; +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.util.common"); +} + namespace lsst::qserv::util { string get_current_host_fqdn(bool all) { @@ -78,4 +86,19 @@ string get_current_host_fqdn(bool all) { return fqdn; } +std::string getCurrentHostFqdnBlocking() { + while (true) { + try { + string result = util::get_current_host_fqdn(); + if (!result.empty()) { + return result; + } + LOGS(_log, LOG_LVL_ERROR, __func__ << " Empty response for the worker hosts's FQDN."); + } catch (std::runtime_error const& ex) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " Failed to obtain worker hosts's FQDN, ex: " << ex.what()); + } + sleep(1); + } +} + } // namespace lsst::qserv::util diff --git a/src/util/common.h b/src/util/common.h index faf7e837a7..6f94678a8a 100644 --- a/src/util/common.h +++ b/src/util/common.h @@ -55,6 +55,12 @@ namespace lsst::qserv::util { */ std::string get_current_host_fqdn(bool all = false); +/** Call get_current_host_fqdn(false) repeatedly until a name is gathered. + * Log messages will be printed. + * It will block until successful. + */ +std::string getCurrentHostFqdnBlocking(); + template typename Map::mapped_type const& getFromMap(Map const& m, typename Map::key_type const& key, typename Map::mapped_type const& defValue) { diff --git a/src/util/testFileNameParser.cc b/src/util/testFileNameParser.cc new file mode 100644 index 0000000000..407fa12921 --- /dev/null +++ b/src/util/testFileNameParser.cc @@ -0,0 +1,110 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include +#include + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "qmeta/types.h" +#include "util/ResultFileNameParser.h" + +// Boost unit test header +#define BOOST_TEST_MODULE ResultFileNameParser +#include + +// Third party headers +#include "boost/filesystem.hpp" + +namespace fs = boost::filesystem; +namespace qmeta = lsst::qserv::qmeta; +namespace test = boost::test_tools; +namespace util = lsst::qserv::util; + +BOOST_AUTO_TEST_SUITE(Suite) + +BOOST_AUTO_TEST_CASE(ResultFileNameParserTest) { + LOGS_INFO("ResultFileNameParserTest"); + + util::ResultFileNameParser fileExpected; + fileExpected.czarId = 1; + fileExpected.queryId = 2; + fileExpected.jobId = 3; + fileExpected.chunkId = 4; + std::string const fileNameNoExt = + std::to_string(fileExpected.czarId) + "-" + std::to_string(fileExpected.queryId) + "-" + + std::to_string(fileExpected.jobId) + "-" + std::to_string(fileExpected.chunkId); + + std::string const fileName = fileNameNoExt + util::ResultFileNameParser::fileExt; + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file(fileNameNoExt); + BOOST_CHECK_EQUAL(file, fileExpected); + BOOST_CHECK_EQUAL(file.czarId, fileExpected.czarId); + BOOST_CHECK_EQUAL(file.queryId, fileExpected.queryId); + BOOST_CHECK_EQUAL(file.jobId, fileExpected.jobId); + BOOST_CHECK_EQUAL(file.chunkId, fileExpected.chunkId); + }); + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file(fileName); + BOOST_CHECK_EQUAL(file, fileExpected); + }); + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file{fs::path(fileName)}; + BOOST_CHECK_EQUAL(file, fileExpected); + }); + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file("/" + fileName); + BOOST_CHECK_EQUAL(file, fileExpected); + }); + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file("/base/" + fileName); + BOOST_CHECK_EQUAL(file, fileExpected); + }); + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file("base/" + fileName); + BOOST_CHECK_EQUAL(file, fileExpected); + }); + + BOOST_CHECK_NO_THROW({ + util::ResultFileNameParser const file(fs::path("/base/") / fileName); + BOOST_CHECK_EQUAL(file, fileExpected); + }); + + BOOST_CHECK_THROW( + { util::ResultFileNameParser const file("1-2-3-4" + fileName); }, std::invalid_argument); + + BOOST_CHECK_THROW( + { util::ResultFileNameParser const file("a-2-3-4-5" + fileName); }, std::invalid_argument); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index 38eaf9ab57..f0583cfe07 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -298,11 +298,11 @@ FileChannelShared::FileChannelShared(std::shared_ptr const& _workerId(workerId), _protobufArena(make_unique()), _scsId(scsSeqId++) { - LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created scsId=" << _scsId << " ujId=" << _uberJobId); + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared created scsId=" << _scsId << " ujId=" << _uberJobId); } FileChannelShared::~FileChannelShared() { - LOGS(_log, LOG_LVL_DEBUG, "~FileChannelShared scsId=" << _scsId << " ujId=" << _uberJobId); + LOGS(_log, LOG_LVL_TRACE, "~FileChannelShared scsId=" << _scsId << " ujId=" << _uberJobId); // Normally, the channel should not be dead at this time. If it's already // dead it means there was a problem to process a query or send back a response // to Czar. In either case, the file would be useless and it has to be deleted @@ -315,7 +315,6 @@ FileChannelShared::~FileChannelShared() { if (isDead() && !_rowLimitComplete) { _removeFile(lock_guard(_tMtx)); } - LOGS(_log, LOG_LVL_DEBUG, "~FileChannelShared end"); } void FileChannelShared::setTaskCount(int taskCount) { _taskCount = taskCount; } @@ -356,17 +355,17 @@ bool FileChannelShared::isRowLimitComplete() const { return _rowLimitComplete; } -bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared_ptr const& task, +void FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared_ptr const& task, bool cancelled) { lock_guard const tMtxLock(_tMtx); if (_rowLimitComplete) { LOGS(_log, LOG_LVL_WARN, __func__ << " already enough rows, this call likely a side effect" << task->getIdStr()); - return false; + return; } // Delete the result file as nobody will come looking for it. _kill(tMtxLock, " buildAndTransmitError"); - return _uberJobData->responseError(multiErr, task, cancelled); + _uberJobData->responseError(multiErr, task->getChunkId(), cancelled, task->getLvlET()); } bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& task, @@ -378,7 +377,7 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetMaxTableSize(); // Fail the operation if the amount of data in the result set exceeds the requested // "large result" limit (in case one was specified). - if (maxTableSize > 0 && bytesTransmitted > maxTableSize) { - string const err = "The result set size " + to_string(bytesTransmitted) + + LOGS(_log, LOG_LVL_TRACE, "bytesWritten=" << _bytesWritten << " max=" << maxTableSize); + if (maxTableSize > 0 && _bytesWritten > maxTableSize) { + string const err = "The result set size " + to_string(_bytesWritten) + " of a job exceeds the requested limit of " + to_string(maxTableSize) + " bytes, task: " + task->getIdStr(); multiErr.push_back(util::Error(util::ErrorCode::WORKER_RESULT_TOO_LARGE, err)); LOGS(_log, LOG_LVL_ERROR, err); erred = true; + return erred; } int const ujRowLimit = task->getRowLimit(); @@ -442,12 +444,14 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrcName(__func__) << " sending start"); if (!_sendResponse(tMtxLockA, task, cancelled, multiErr, rowLimitComplete)) { LOGS(_log, LOG_LVL_ERROR, "Could not transmit the request completion message to Czar."); erred = true; } else { LOGS(_log, LOG_LVL_TRACE, __func__ << " " << task->getIdStr() << " sending done!!!"); } + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared " << task->cName(__func__) << " sending done!!!"); } } transmitT.stop(); @@ -474,10 +478,10 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& streamMutexLock, string const& note) { - LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared::" << __func__ << " " << note); + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared::" << __func__ << " " << note); bool oldVal = _dead.exchange(true); if (!oldVal) { - LOGS(_log, LOG_LVL_WARN, "FileChannelShared first kill call " << note); + LOGS(_log, LOG_LVL_WARN, "FileChannelShared::" << __func__ << " first kill call " << note); } return oldVal; } @@ -486,7 +490,7 @@ void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p MYSQL_RES* mResult, uint64_t& bytes, uint32_t& rows, util::MultiError& multiErr) { if (!_file.is_open()) { - _fileName = task->resultFileAbsPath(); + _fileName = task->getUberJobData()->resultFilePath(); _file.open(_fileName, ios::out | ios::trunc | ios::binary); if (!(_file.is_open() && _file.good())) { throw runtime_error("FileChannelShared::" + string(__func__) + @@ -556,7 +560,7 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ lock_guard const streamMutexLock(_streamMutex); QSERV_LOGCONTEXT_QUERY_JOB(queryId, jobId); - LOGS(_log, LOG_LVL_DEBUG, __func__); + if (isDead() && !mustSend) { LOGS(_log, LOG_LVL_INFO, __func__ << ": aborting transmit since sendChannel is dead."); return false; @@ -564,7 +568,7 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ // Prepare the response object and serialize in into a message that will // be sent to the Czar. - string httpFileUrl = task->resultFileHttpUrl(); + string httpFileUrl = task->getUberJobData()->resultFileHttpUrl(); _uberJobData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); return true; } diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 649bbae374..a9b5297db6 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -145,8 +145,8 @@ class FileChannelShared { /// @return true if this is the first time this function has been called. bool getFirstChannelSqlConn() { return _firstChannelSqlConn.exchange(false); } - /// @return a transmit data object indicating the errors in 'multiErr'. - bool buildAndTransmitError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); + /// Build and transmit a transmit data object indicating the errors in 'multiErr'. + void buildAndTransmitError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); /// Extract the SQL results and write them into the file and notify Czar after the last /// row of the result result set depending on theis channel has been processed. @@ -284,6 +284,8 @@ class FileChannelShared { /// much faster to answer the query without scanning all 1000 chunks. std::atomic _rowLimitComplete; std::atomic _dead{false}; ///< Set to true when the contents of the file are no longer useful. + + std::atomic _bytesWritten{0}; ///< Total bytes written. }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 55e29043f3..ceb6f06296 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -37,7 +37,6 @@ // Third-party headers #include -#include "boost/filesystem.hpp" // LSST headers #include "lsst/log/Log.h" @@ -67,31 +66,17 @@ using namespace std; using namespace std::chrono_literals; using namespace nlohmann; -namespace fs = boost::filesystem; namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.Task"); -string buildUjResultFilePath(lsst::qserv::wbase::UberJobData::Ptr const& ujData, - string const& resultsDirname) { - if (resultsDirname.empty()) return resultsDirname; - fs::path path(resultsDirname); - // UberJobs have multiple chunks which can each have different attempt numbers. - // However, each CzarID + UberJobId should be unique as UberJobs are not retried. - path /= to_string(ujData->getCzarId()) + "-" + to_string(ujData->getQueryId()) + "-" + - to_string(ujData->getUberJobId()) + "-0" + ".proto"; - return path.string(); -} - size_t const MB_SIZE_BYTES = 1024 * 1024; } // namespace namespace lsst::qserv::wbase { -string const Task::_fqdn = util::get_current_host_fqdn(); - // Task::ChunkEqual functor bool Task::ChunkEqual::operator()(Task::Ptr const& x, Task::Ptr const& y) { if (!x || !y) { @@ -126,11 +111,12 @@ atomic taskSequence{0}; ///< Unique identifier source for Task. /// the util::CommandThreadPool is not called here. Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, size_t templateId, bool hasSubchunks, int subchunkId, string const& db, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSize, vector const& fragSubTables, vector const& fragSubchunkIds, shared_ptr const& sc, - std::shared_ptr const& queryStats_, uint16_t resultsHttpPort) - : _sendChannel(sc), + std::shared_ptr const& queryStats_) + : _logLvlWT(LOG_LVL_WARN), + _logLvlET(LOG_LVL_ERROR), + _sendChannel(sc), _tSeq(++taskSequence), _qId(ujData->getQueryId()), _templateId(templateId), @@ -143,24 +129,10 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun _fragmentHasSubchunks(hasSubchunks), _db(db), _czarId(ujData->getCzarId()), - _scanInfo(scanInfo), - _scanInteractive(scanInteractive), _queryStats(queryStats_), - _maxTableSize(maxTableSize * ::MB_SIZE_BYTES), - _rowLimit(ujData->getRowLimit()) { - // These attributes will be passed back to Czar in the Protobuf response - // to advice which result delivery channel to use. - auto const workerConfig = wconfig::WorkerConfig::instance(); - auto const resultDeliveryProtocol = workerConfig->resultDeliveryProtocol(); - _resultFilePath = ::buildUjResultFilePath(ujData, workerConfig->resultsDirname()); - auto const fqdn = util::get_current_host_fqdn(); - if (resultDeliveryProtocol == wconfig::ConfigValResultDeliveryProtocol::HTTP) { - // TODO:UJ it seems like this should just be part of the FileChannelShared??? - _resultFileHttpUrl = "http://" + fqdn + ":" + to_string(resultsHttpPort) + _resultFilePath; - } else { - throw runtime_error("wbase::Task::Task: unsupported results delivery protocol: " + - wconfig::ConfigValResultDeliveryProtocol::toString(resultDeliveryProtocol)); - } + _rowLimit(ujData->getRowLimit()), + _ujData(ujData), + _idStr(ujData->getIdStr() + " jId=" + to_string(_jId) + " sc=" + to_string(_subchunkId)) { user = defaultUser; // Create sets and vectors for 'aquiring' subchunk temporary tables. @@ -170,12 +142,13 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun if (!_fragmentHasSubchunks) { /// FUTURE: Why acquire anything if there are no subchunks in the fragment? /// This branch never seems to happen, but this needs to be proven beyond any doubt. + auto scanInfo = _ujData->getScanInfo(); for (auto const& scanTbl : scanInfo->infoTables) { dbTbls_.emplace(scanTbl.db, scanTbl.table); - LOGS(_log, LOG_LVL_INFO, + LOGS(_log, LOG_LVL_TRACE, "Task::Task scanTbl.db=" << scanTbl.db << " scanTbl.table=" << scanTbl.table); } - LOGS(_log, LOG_LVL_INFO, + LOGS(_log, LOG_LVL_TRACE, "fragment a db=" << _db << ":" << _chunkId << " dbTbls=" << util::printable(dbTbls_)); } else { for (TaskDbTbl const& fDbTbl : fragSubTables) { @@ -188,137 +161,29 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun } subchunksVect_ = fragSubchunkIds; - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "fragment b db=" << _db << ":" << _chunkId << " dbTableSet" << util::printable(dbTbls_) << " subChunks=" << util::printable(subchunksVect_)); } _dbTblsAndSubchunks = make_unique(dbTbls_, subchunksVect_); + + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " created"); } Task::~Task() {} -/* &&& -std::vector Task::createTasksForChunk( - std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, - std::shared_ptr const& sqlConnMgr, - std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { - QueryId qId = ujData->getQueryId(); - UberJobId ujId = ujData->getUberJobId(); - CzarIdType czId = ujData->getCzarId(); - - wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); - UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); - - string funcN(__func__); - funcN += " QID=" + to_string(qId) + " "; - - vector vect; - for (auto const& job : jsJobs) { - json const& jsJobDesc = job["jobdesc"]; - http::RequestBodyJSON rbJobDesc(jsJobDesc); - // See qproc::TaskMsgFactory::makeMsgJson for message construction. - auto const jdCzarId = rbJobDesc.required("czarId"); - auto const jdQueryId = rbJobDesc.required("queryId"); - if (jdQueryId != qId) { - throw TaskException(ERR_LOC, string("ujId=") + to_string(ujId) + " qId=" + to_string(qId) + - " QueryId mismatch Job qId=" + to_string(jdQueryId)); - } - auto const jdJobId = rbJobDesc.required("jobId"); - auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - auto const jdScanPriority = rbJobDesc.required("scanPriority"); - auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); - auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); - auto const jdChunkId = rbJobDesc.required("chunkId"); - LOGS(_log, LOG_LVL_TRACE, - funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId - << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive - << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); - - auto const jdQueryFragments = rbJobDesc.required("queryFragments"); - int fragmentNumber = 0; - for (auto const& frag : jdQueryFragments) { - vector fragSubQueries; - vector fragSubchunkIds; - vector fragSubTables; - LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); - http::RequestBodyJSON rbFrag(frag); - auto const& jsQueries = rbFrag.required("queries"); - // TODO:UJ move to uberjob???, these should be the same for all jobs - for (auto const& subQ : jsQueries) { - http::RequestBodyJSON rbSubQ(subQ); - auto const subQuery = rbSubQ.required("subQuery"); - LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); - fragSubQueries.push_back(subQuery); - } - auto const& resultTable = rbFrag.required("resultTable"); - auto const& jsSubIds = rbFrag.required("subchunkIds"); - for (auto const& scId : jsSubIds) { - fragSubchunkIds.push_back(scId); - } - auto const& jsSubTables = rbFrag.required("subchunkTables"); - - for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? - http::RequestBodyJSON rbScDbTable(scDbTable); - string scDb = rbScDbTable.required("scDb"); - string scTable = rbScDbTable.required("scTable"); - TaskDbTbl scDbTbl(scDb, scTable); - fragSubTables.push_back(scDbTbl); - } - - for (string const& fragSubQ : fragSubQueries) { - size_t templateId = userQueryInfo->addTemplate(fragSubQ); - if (fragSubchunkIds.empty()) { - bool const noSubchunks = false; - int const subchunkId = -1; - auto task = Task::Ptr(new Task( - ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, templateId, - noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, - fragSubTables, fragSubchunkIds, sendChannel, queryStats, resultsHttpPort)); - - vect.push_back(task); - } else { - for (auto subchunkId : fragSubchunkIds) { - bool const hasSubchunks = true; - auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, - fragmentNumber, templateId, hasSubchunks, subchunkId, - jdQuerySpecDb, scanInfo, scanInteractive, - maxTableSizeMb, fragSubTables, fragSubchunkIds, - sendChannel, queryStats, resultsHttpPort)); - vect.push_back(task); - } - } - } - ++fragmentNumber; - } - } - - for (auto taskPtr : vect) { - // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. - taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, - sqlConnMgr, queriesAndChunks)); - } - return vect; -} -*/ - std::vector Task::createTasksFromUberJobMsg( std::shared_ptr const& ujMsg, std::shared_ptr const& ujData, std::shared_ptr const& sendChannel, std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, - std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { + std::shared_ptr const& queriesAndChunks) { QueryId qId = ujData->getQueryId(); UberJobId ujId = ujData->getUberJobId(); CzarIdType czId = ujData->getCzarId(); vector vect; // List of created tasks to be returned. - wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); @@ -340,19 +205,17 @@ std::vector Task::createTasksFromUberJobMsg( auto jobSubQueryTempMap = ujMsg->getJobSubQueryTempMap(); auto jobDbTablesMap = ujMsg->getJobDbTablesMap(); auto jobMsgVect = ujMsg->getJobMsgVect(); - int maxTableSizeMb = ujMsg->getMaxTableSizeMb(); - auto scanInfo = ujMsg->getScanInfo(); for (auto const& jobMsg : *jobMsgVect) { JobId jobId = jobMsg->getJobId(); int attemptCount = jobMsg->getAttemptCount(); std::string chunkQuerySpecDb = jobMsg->getChunkQuerySpecDb(); - bool scanInteractive = jobMsg->getScanInteractive(); int chunkId = jobMsg->getChunkId(); std::vector chunkScanTableIndexes = jobMsg->getChunkScanTableIndexes(); auto jobFragments = jobMsg->getJobFragments(); int fragmentNumber = 0; + for (auto const& fMsg : *jobFragments) { // These need to be constructed for the fragment vector fragSubQueries; @@ -379,20 +242,18 @@ std::vector Task::createTasksFromUberJobMsg( if (fragSubchunkIds.empty()) { bool const noSubchunks = false; int const subchunkId = -1; - auto task = Task::Ptr(new Task( - ujData, jobId, attemptCount, chunkId, fragmentNumber, templateId, noSubchunks, - subchunkId, chunkQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, - fragSubTables, fragSubchunkIds, sendChannel, queryStats, resultsHttpPort)); + auto task = Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, + templateId, noSubchunks, subchunkId, chunkQuerySpecDb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats)); vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; - auto task = Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, - templateId, hasSubchunks, subchunkId, chunkQuerySpecDb, - scanInfo, scanInteractive, maxTableSizeMb, - fragSubTables, fragSubchunkIds, sendChannel, - queryStats, resultsHttpPort)); + auto task = + Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, + templateId, hasSubchunks, subchunkId, chunkQuerySpecDb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats)); vect.push_back(task); } } @@ -406,19 +267,19 @@ std::vector Task::createTasksFromUberJobMsg( taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, sqlConnMgr, queriesAndChunks)); } + return vect; } std::vector Task::createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& sendChannel, int maxTableSizeMb, std::shared_ptr const& chunkResourceMgr) { QueryId qId = ujData->getQueryId(); UberJobId ujId = ujData->getUberJobId(); CzarIdType czId = ujData->getCzarId(); string funcN(__func__); - funcN += " QID=" + to_string(qId) + " "; + funcN += " QID=" + to_string(qId) + " czId=" + to_string(czId); vector vect; for (auto const& job : jsJobs) { @@ -434,14 +295,11 @@ std::vector Task::createTasksForUnitTest( auto const jdJobId = rbJobDesc.required("jobId"); auto const jdAttemptCount = rbJobDesc.required("attemptCount"); auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - auto const jdScanPriority = rbJobDesc.required("scanPriority"); - auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); auto const jdChunkId = rbJobDesc.required("chunkId"); LOGS(_log, LOG_LVL_TRACE, funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); auto const jdQueryFragments = rbJobDesc.required("queryFragments"); @@ -476,23 +334,22 @@ std::vector Task::createTasksForUnitTest( } for (string const& fragSubQ : fragSubQueries) { + LOGS(_log, LOG_LVL_DEBUG, "ignoring " << fragSubQ); if (fragSubchunkIds.empty()) { bool const noSubchunks = false; int const subchunkId = -1; auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, - 0, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, - scanInteractive, maxTableSizeMb, fragSubTables, - fragSubchunkIds, sendChannel, nullptr, 0)); + 0, noSubchunks, subchunkId, jdQuerySpecDb, fragSubTables, + fragSubchunkIds, sendChannel, nullptr)); vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; - auto task = Task::Ptr(new Task( - ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, 0, hasSubchunks, - subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, - fragSubTables, fragSubchunkIds, sendChannel, nullptr, 0)); - + auto task = + Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, + 0, hasSubchunks, subchunkId, jdQuerySpecDb, fragSubTables, + fragSubchunkIds, sendChannel, nullptr)); vect.push_back(task); } } @@ -504,6 +361,10 @@ std::vector Task::createTasksForUnitTest( return vect; } +protojson::ScanInfo::Ptr Task::getScanInfo() const { return _ujData->getScanInfo(); } + +bool Task::getScanInteractive() const { return _ujData->getScanInteractive(); } + void Task::action(util::CmdData* data) { string tIdStr = getIdStr(); if (_queryStarted.exchange(true)) { @@ -521,23 +382,26 @@ void Task::action(util::CmdData* data) { // Get a local copy for safety. auto qr = _taskQueryRunner; bool success = false; + string errStr; try { success = qr->runQuery(); } catch (UnsupportedError const& e) { LOGS(_log, LOG_LVL_ERROR, __func__ << " runQuery threw UnsupportedError " << e.what() << tIdStr); + errStr = e.what(); } if (not success) { - LOGS(_log, LOG_LVL_ERROR, "runQuery failed " << tIdStr); - if (not getSendChannel()->kill("Foreman::_setRunFunc")) { - LOGS(_log, LOG_LVL_WARN, "runQuery sendChannel already killed " << tIdStr); + LOGS(_log, _logLvlET, "runQuery failed " << tIdStr); + if (not getSendChannel()->kill("Task::action")) { + LOGS(_log, _logLvlWT, "runQuery sendChannel already killed " << tIdStr); } + // Send a message back saying this UberJobFailed, redundant error messages should be + // harmless. + util::MultiError multiErr; + bool logLvl = (_logLvlET != LOG_LVL_TRACE); + util::Error err(_chunkId, string("UberJob run error ") + errStr, util::ErrorCode::NONE, logLvl); + multiErr.push_back(err); + _ujData->responseError(multiErr, -1, false, _logLvlET); } - - // The QueryRunner class access to sendChannel for results is over by this point. - // 'task' contains statistics that are still useful. However, the resources used - // by sendChannel need to be freed quickly. - LOGS(_log, LOG_LVL_DEBUG, __func__ << " calling resetSendChannel() for " << tIdStr); - resetSendChannel(); // Frees the SendChannel instance } string Task::getQueryString() const { @@ -570,20 +434,14 @@ void Task::cancel() { return; } - util::HoldTrack::Mark markA(ERR_LOC, "Task::cancel"); LOGS(_log, LOG_LVL_DEBUG, "Task::cancel " << getIdStr()); auto qr = _taskQueryRunner; // Need a copy in case _taskQueryRunner is reset. if (qr != nullptr) { qr->cancel(); } - // At this point, this code doesn't do anything. It may be - // useful to remove this task from the scheduler, but it - // seems doubtful that that would improve performance. - auto sched = _taskScheduler.lock(); - if (sched != nullptr) { - sched->taskCancelled(this); - } + _logLvlWT = LOG_LVL_TRACE; + _logLvlET = LOG_LVL_TRACE; } bool Task::checkCancelled() { @@ -631,21 +489,21 @@ bool Task::isRunning() const { } void Task::started(chrono::system_clock::time_point const& now) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << getIdStr() << " started"); + LOGS(_log, LOG_LVL_TRACE, __func__ << " " << getIdStr() << " started"); lock_guard guard(_stateMtx); _state = TaskState::STARTED; _startTime = now; } void Task::queryExecutionStarted() { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << getIdStr() << " executing"); + LOGS(_log, LOG_LVL_TRACE, __func__ << " " << getIdStr() << " executing"); lock_guard guard(_stateMtx); _state = TaskState::EXECUTING_QUERY; _queryExecTime = chrono::system_clock::now(); } void Task::queried() { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << getIdStr() << " reading"); + LOGS(_log, LOG_LVL_TRACE, __func__ << " " << getIdStr() << " reading"); lock_guard guard(_stateMtx); _state = TaskState::READING_DATA; _queryTime = chrono::system_clock::now(); @@ -657,7 +515,7 @@ void Task::queried() { /// Set values associated with the Task being finished. /// @return milliseconds to complete the Task, system clock time. chrono::milliseconds Task::finished(chrono::system_clock::time_point const& now) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " " << getIdStr() << " finished"); + LOGS(_log, LOG_LVL_TRACE, __func__ << " " << getIdStr() << " finished"); chrono::milliseconds duration; { lock_guard guard(_stateMtx); @@ -669,7 +527,7 @@ chrono::milliseconds Task::finished(chrono::system_clock::time_point const& now) if (duration.count() < 1) { duration = chrono::milliseconds{1}; } - LOGS(_log, LOG_LVL_DEBUG, "processing millisecs=" << duration.count()); + LOGS(_log, LOG_LVL_TRACE, "processing millisecs=" << duration.count()); return duration; } @@ -707,8 +565,7 @@ nlohmann::json Task::getJson() const { js["fragmentId"] = _queryFragmentNum; js["attemptId"] = _attemptCount; js["sequenceId"] = _tSeq; - js["scanInteractive"] = _scanInteractive; - js["maxTableSize"] = _maxTableSize; + js["maxTableSize"] = _ujData->getMaxTableSizeBytes(); js["cancelled"] = to_string(_cancelled); js["state"] = static_cast(_state.load()); js["createTime_msec"] = util::TimeUtils::tp2ms(_createTime); @@ -726,6 +583,8 @@ nlohmann::json Task::getJson() const { return js; } +int64_t Task::getMaxTableSize() const { return _ujData->getMaxTableSizeBytes(); } + ostream& operator<<(ostream& os, Task const& t) { os << "Task: " << "msg: " << t.getIdStr() << " chunk=" << t._chunkId << " db=" << t._db << " " << t.getQueryString(); diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 1f15e7dc64..4c20cebed0 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -148,62 +148,41 @@ class Task : public util::CommandForThreadPool { bool operator()(Ptr const& x, Ptr const& y); }; - std::string cName(const char* func) const { return std::string("Task::") + func; } + std::string cName(const char* func) const { return std::string("Task::") + func + " " + _idStr; } // TODO:UJ too many parameters. // - fragmentNumber seems pointless // - hasSubchunks seems redundant. // Hopefully, many are the same for all tasks and can be moved to ujData and userQueryInfo. - // Candidates: scanInfo, maxTableSizeMb, FileChannelShared, resultsHttpPort. + // Candidates: maxTableSizeMb, FileChannelShared, resultsHttpPort. // Unfortunately, this will be much easier if it is done after xrootd method is removed. Task(std::shared_ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, size_t templateId, bool hasSubchunks, int subchunkId, std::string const& db, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, std::vector const& fragSubTables, std::vector const& fragSubchunkIds, std::shared_ptr const& sc, - std::shared_ptr const& queryStats_, uint16_t resultsHttpPort = 8080); + std::shared_ptr const& queryStats_); Task& operator=(const Task&) = delete; Task(const Task&) = delete; virtual ~Task(); -/* &&& - /// Read json to generate a vector of one or more task for a chunk. - static std::vector createTasksForChunk( /// &&& delete - std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr, - mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, - std::shared_ptr const& queriesAndChunks, - uint16_t resultsHttpPort = 8080); -*/ - - /// &&& + /// Create the Tasks needed to run an UberJob on this worker. static std::vector createTasksFromUberJobMsg( std::shared_ptr const& uberJobMsg, std::shared_ptr const& ujData, std::shared_ptr const& sendChannel, std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, - std::shared_ptr const& queriesAndChunks, - uint16_t resultsHttpPort = 8080); + std::shared_ptr const& queriesAndChunks); - //&&& + /// Create Tasks needed to run unit tests. static std::vector createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr - //&&&mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& - // sqlConnMgr, - //&&&std::shared_ptr const& queriesAndChunks, - //&&&uint16_t resultsHttpPort = 8080); - ); + std::shared_ptr const& sendChannel, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr); std::shared_ptr getSendChannel() const { return _sendChannel; } - void resetSendChannel() { _sendChannel.reset(); } ///< reset the shared pointer for FileChannelShared - std::string user; ///< Incoming username + std::string user; ///< Incoming username // Note that manpage spec of "26 bytes" is insufficient /// This is the function the scheduler will run, overriden from the util::Command class. @@ -232,8 +211,7 @@ class Task : public util::CommandForThreadPool { TaskState state() const { return _state; } std::string getQueryString() const; - std::string const& resultFileAbsPath() const { return _resultFileAbsPath; } - std::string const& resultFileHttpUrl() const { return _resultFileHttpUrl; } + bool setTaskQueryRunner( TaskQueryRunner::Ptr const& taskQueryRunner); ///< return true if already cancelled. void freeTaskQueryRunner(TaskQueryRunner* tqr); @@ -251,9 +229,10 @@ class Task : public util::CommandForThreadPool { size_t getTemplateId() const { return _templateId; } int getJobId() const { return _jId; } int getAttemptCount() const { return _attemptCount; } - bool getScanInteractive() { return _scanInteractive; } - int64_t getMaxTableSize() const { return _maxTableSize; } - protojson::ScanInfo::Ptr getScanInfo() { return _scanInfo; } + bool getScanInteractive() const; + int64_t getMaxTableSize() const; + + protojson::ScanInfo::Ptr getScanInfo() const; void setOnInteractive(bool val) { _onInteractive = val; } bool getOnInteractive() { return _onInteractive; } @@ -326,12 +305,20 @@ class Task : public util::CommandForThreadPool { setFunc(func); } + std::shared_ptr getUberJobData() const { return _ujData; } + /// Returns the LIMIT of rows for the query enforceable at the worker, where values <= 0 indicate /// that there is no limit to the number of rows sent back by the worker. /// @see UberJobData::getRowLimit() int getRowLimit() { return _rowLimit; } + int getLvlWT() const { return _logLvlWT; } + int getLvlET() const { return _logLvlET; } + private: + std::atomic _logLvlWT; ///< Normally LOG_LVL_WARN, set to TRACE in cancelled Tasks. + std::atomic _logLvlET; ///< Normally LOG_LVL_ERROR, set to TRACE in cancelled Tasks. + std::shared_ptr _sendChannel; ///< Send channel. uint64_t const _tSeq = 0; ///< identifier for the specific task @@ -350,21 +337,10 @@ class Task : public util::CommandForThreadPool { /// Set of tables and vector of subchunk ids used by ChunkResourceRequest. Do not change/reset. std::unique_ptr _dbTblsAndSubchunks; - /// The path to the result file. - std::string _resultFileAbsPath; - - /// The name of the result file. - std::string _resultFileName; - - /// The HTTP URL for the result file: "http://:/" + _resultFileName - std::string _resultFileHttpUrl; - std::atomic _queryStarted{false}; ///< Set to true when the query is about to be run. std::atomic _cancelled{false}; TaskQueryRunner::Ptr _taskQueryRunner; std::weak_ptr _taskScheduler; - protojson::ScanInfo::Ptr _scanInfo; - bool _scanInteractive; ///< True if the czar thinks this query should be interactive. bool _onInteractive{ false}; ///< True if the scheduler put this task on the interactive (group) scheduler. @@ -394,7 +370,10 @@ class Task : public util::CommandForThreadPool { /// When > 0, indicates maximum number of rows needed for a result. int const _rowLimit; - bool _unitTest = false; ///< + std::shared_ptr _ujData; + std::string const _idStr; + + bool _unitTest = false; ///< Only true in unit tests. }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index a70793f2a4..651046474c 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -26,6 +26,7 @@ // System headers // Third party headers +#include "boost/filesystem.hpp" // LSST headers #include "lsst/log/Log.h" @@ -39,6 +40,7 @@ #include "http/RequestQuery.h" #include "util/Bug.h" #include "util/MultiError.h" +#include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" #include "wpublish/ChunkInventory.h" #include "wpublish/QueriesAndChunks.h" @@ -46,6 +48,8 @@ using namespace std; using namespace nlohmann; +namespace fs = boost::filesystem; + namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.UberJobData"); @@ -56,8 +60,10 @@ namespace lsst::qserv::wbase { UberJobData::UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, int czarPort, uint64_t queryId, int rowLimit, - std::string const& workerId, std::shared_ptr const& foreman, - std::string const& authKey) + uint64_t maxTableSizeBytes, protojson::ScanInfo::Ptr const& scanInfo, + bool scanInteractive, std::string const& workerId, + std::shared_ptr const& foreman, std::string const& authKey, + uint16_t resultsHttpPort) : _uberJobId(uberJobId), _czarName(czarName), _czarId(czarId), @@ -65,10 +71,14 @@ UberJobData::UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta _czarPort(czarPort), _queryId(queryId), _rowLimit(rowLimit), + _maxTableSizeBytes(maxTableSizeBytes), _workerId(workerId), _authKey(authKey), + _resultsHttpPort(resultsHttpPort), _foreman(foreman), - _idStr(string("QID=") + to_string(_queryId) + ":ujId=" + to_string(_uberJobId)) {} + _scanInteractive(scanInteractive), + _scanInfo(scanInfo), + _idStr(string("QID=") + to_string(_queryId) + "_ujId=" + to_string(_uberJobId)) {} void UberJobData::setFileChannelShared(std::shared_ptr const& fileChannelShared) { if (_fileChannelShared != nullptr && _fileChannelShared != fileChannelShared) { @@ -79,10 +89,17 @@ void UberJobData::setFileChannelShared(std::shared_ptr const& void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, uint64_t headerCount) { - LOGS(_log, LOG_LVL_TRACE, + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize << " headerCount=" << headerCount); + // Latch to prevent errors from being transmitted. + // NOTE: Calls to responseError() and responseFileReady() are protected by the + // mutex in FileChannelShared (_tMtx). + if (_responseState.exchange(SENDING_FILEURL) != NOTHING) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " _responseState was " << _responseState << " instead of NOTHING"); + } string workerIdStr; if (_foreman != nullptr) { workerIdStr = _foreman->chunkInventory()->id(); @@ -112,9 +129,19 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount _queueUJResponse(method, headers, url, requestContext, requestStr); } -bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& task, - bool cancelled) { - LOGS(_log, LOG_LVL_INFO, cName(__func__)); +void UberJobData::responseError(util::MultiError& multiErr, int chunkId, bool cancelled, int logLvl) { + // TODO:UJ Maybe register this UberJob as failed with a czar notification method + // so that a secondary means can be used to make certain the czar hears about + // the error. See related TODO:UJ comment in responseFileReady() + LOGS(_log, logLvl, cName(__func__)); + // NOTE: Calls to responseError() and responseFileReady() are protected by the + // mutex in FileChannelShared (_tMtx). + if (_responseState == NOTHING) { + _responseState = SENDING_ERROR; + } else { + LOGS(_log, logLvl, cName(__func__) << " Already sending a different message."); + return; + } string errorMsg; int errorCode = 0; if (!multiErr.empty()) { @@ -125,9 +152,8 @@ bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptrgetChunkId()) + - ": " + errorMsg; - LOGS(_log, LOG_LVL_ERROR, errorMsg); + errorMsg = cName(__func__) + " error(s) in result for chunk #" + to_string(chunkId) + ": " + errorMsg; + LOGS(_log, logLvl, errorMsg); } json request = {{"version", http::MetaModule::version}, @@ -146,7 +172,6 @@ bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& headers_, @@ -171,6 +196,30 @@ void UberJobData::_queueUJResponse(http::Method method_, std::vectorresultsDirname(); + if (resultsDirname.empty()) return resultsDirname; + return (fs::path(resultsDirname) / _resultFileName()).string(); +} + +std::string UberJobData::resultFileHttpUrl() const { + auto const workerConfig = wconfig::WorkerConfig::instance(); + auto const resultDeliveryProtocol = workerConfig->resultDeliveryProtocol(); + if (resultDeliveryProtocol != wconfig::ConfigValResultDeliveryProtocol::HTTP) { + throw runtime_error("wbase::Task::Task: unsupported results delivery protocol: " + + wconfig::ConfigValResultDeliveryProtocol::toString(resultDeliveryProtocol)); + } + // TODO:UJ it seems like this should just be part of the FileChannelShared??? + return "http://" + _foreman->getFqdn() + ":" + to_string(_resultsHttpPort) + "/" + _resultFileName(); +} + void UberJobData::cancelAllTasks() { LOGS(_log, LOG_LVL_INFO, cName(__func__)); if (_cancelled.exchange(true) == false) { @@ -183,11 +232,12 @@ void UberJobData::cancelAllTasks() { string UJTransmitCmd::cName(const char* funcN) const { stringstream os; - os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " qId=" << _queryId << " ujId=" << _uberJobId; + os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " QID=" << _queryId << "_ujId=" << _uberJobId; return os.str(); } void UJTransmitCmd::action(util::CmdData* data) { + LOGS(_log, LOG_LVL_TRACE, cName(__func__)); // Make certain _selfPtr is reset before leaving this function. // If a retry is needed, duplicate() is called. class ResetSelf { @@ -228,12 +278,13 @@ void UJTransmitCmd::action(util::CmdData* data) { auto wCzInfo = _foreman->getWCzarInfoMap()->getWCzarInfo(_czarId); // This will check if the czar is believed to be alive and try the queue the query to be tried // again at a lower priority. It it thinks the czar is dead, it will throw it away. - // TODO:UJ &&& I have my doubts about this as a reconnected czar may go down in flames - // &&& as it is hit with thousands of these. - // &&& Alternate plan, set a flag in the status message response (WorkerQueryStatusData) - // &&& indicates some messages failed. When the czar sees the flag, it'll request a - // &&& message from the worker that contains all of the failed transmit data and handle - // &&& that. All of these failed transmits should fit in a single message. + // TODO:UJ I have my doubts about this as a reconnected czar may go down in flames + // as it is hit with thousands of these. The priority queue in the wPool should + // help limit these to sane amounts, but the alternate plan below is probably safer. + // Alternate plan, set a flag in the status message response (WorkerQueryStatusData) + // indicates some messages failed. When the czar sees the flag, it'll request a + // message from the worker that contains all of the failed transmit data and handle + // that. All of these failed transmits should fit in a single message. if (wCzInfo->checkAlive(CLOCK::now())) { auto wPool = _foreman->getWPool(); if (wPool != nullptr) { @@ -256,8 +307,7 @@ void UJTransmitCmd::action(util::CmdData* data) { } void UJTransmitCmd::kill() { - string const funcN("UJTransmitCmd::kill"); - LOGS(_log, LOG_LVL_WARN, funcN); + LOGS(_log, LOG_LVL_WARN, cName(__func__)); auto sPtr = _selfPtr; _selfPtr.reset(); if (sPtr == nullptr) { @@ -266,6 +316,7 @@ void UJTransmitCmd::kill() { } UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { + LOGS(_log, LOG_LVL_INFO, cName(__func__)); auto ujD = _ujData.lock(); if (ujD == nullptr) { return nullptr; diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index d4765fbbe7..703c039b24 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -41,6 +41,10 @@ namespace lsst::qserv { +namespace protojson { +class ScanInfo; +} + namespace util { class MultiError; } @@ -61,20 +65,26 @@ class UberJobData : public std::enable_shared_from_this { public: using Ptr = std::shared_ptr; + enum ResponseState { SENDING_ERROR = -1, NOTHING = 0, SENDING_FILEURL = 1 }; + UberJobData() = delete; UberJobData(UberJobData const&) = delete; static Ptr create(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string const& czarHost, int czarPort, uint64_t queryId, int rowLimit, - std::string const& workerId, std::shared_ptr const& foreman, - std::string const& authKey) { + uint64_t maxTableSizeBytes, std::shared_ptr const& scanInfo, + bool scanInteractive, std::string const& workerId, + std::shared_ptr const& foreman, std::string const& authKey, + uint16_t resultsHttpPort = 8080) { return Ptr(new UberJobData(uberJobId, czarName, czarId, czarHost, czarPort, queryId, rowLimit, - workerId, foreman, authKey)); + maxTableSizeBytes, scanInfo, scanInteractive, workerId, foreman, authKey, + resultsHttpPort)); } /// Set file channel for this UberJob void setFileChannelShared(std::shared_ptr const& fileChannelShared); - void setScanInteractive(bool scanInteractive) { _scanInteractive = scanInteractive; } + bool getScanInteractive() const { return _scanInteractive; } + std::shared_ptr getScanInfo() const { return _scanInfo; } UberJobId getUberJobId() const { return _uberJobId; } qmeta::CzarId getCzarId() const { return _czarId; } @@ -82,6 +92,7 @@ class UberJobData : public std::enable_shared_from_this { int getCzarPort() const { return _czarPort; } uint64_t getQueryId() const { return _queryId; } std::string getWorkerId() const { return _workerId; } + uint64_t getMaxTableSizeBytes() const { return _maxTableSizeBytes; } /// Add the tasks defined in the UberJob to this UberJobData object. void addTasks(std::vector> const& tasks) { @@ -94,9 +105,9 @@ class UberJobData : public std::enable_shared_from_this { uint64_t headerCount); // TODO:UJ remove headerCount /// Let the Czar know there's been a problem. - bool responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); + void responseError(util::MultiError& multiErr, int chunkId, bool cancelled, int logLvl); - std::string getIdStr() const { return _idStr; } + std::string const& getIdStr() const { return _idStr; } std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } bool getCancelled() const { return _cancelled; } @@ -108,12 +119,20 @@ class UberJobData : public std::enable_shared_from_this { /// that there is no limit to the number of rows sent back by the worker. /// Workers can only safely limit rows for queries that have the LIMIT clause without other related /// clauses like ORDER BY. - int getRowLimit() { return _rowLimit; } + int getRowLimit() const { return _rowLimit; } + + std::string resultFilePath() const; + std::string resultFileHttpUrl() const; private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, - int czarPort, uint64_t queryId, int rowLimit, std::string const& workerId, - std::shared_ptr const& foreman, std::string const& authKey); + int czarPort, uint64_t queryId, int rowLimit, uint64_t maxTableSizeBytes, + std::shared_ptr const& scanInfo, bool scanInteractive, + std::string const& workerId, std::shared_ptr const& foreman, + std::string const& authKey, uint16_t resultsHttpPort); + + /// Return the name of the file that will contain the results of the query. + std::string _resultFileName() const; /// Queue the response to be sent to the originating czar. void _queueUJResponse(http::Method method_, std::vector const& headers_, @@ -127,8 +146,10 @@ class UberJobData : public std::enable_shared_from_this { int const _czarPort; QueryId const _queryId; int const _rowLimit; ///< If > 0, only read this many rows before return the results. + uint64_t const _maxTableSizeBytes; std::string const _workerId; std::string const _authKey; + uint16_t const _resultsHttpPort; ///< = 8080 std::shared_ptr const _foreman; @@ -137,12 +158,21 @@ class UberJobData : public std::enable_shared_from_this { std::mutex _ujTasksMtx; ///< Protects _ujTasks. - std::string const _idStr; - /// True if this an interactive (aka high priority) user query. std::atomic _scanInteractive; + /// Pointer to scan rating and table information. + std::shared_ptr _scanInfo; + + std::string const _idStr; + std::atomic _cancelled{false}; ///< Set to true if this was cancelled. + + /// Either a file ULR or error needs to be sent back to the czar. + /// In the case of LIMIT queries, once a file URL has been sent, + /// the system must be prevented from sending errors back to the czar + /// for Tasks that were cancelled due to the LIMIT already being reached. + std::atomic _responseState{NOTHING}; }; /// This class puts the information about a locally finished UberJob into a command diff --git a/src/wconfig/WorkerConfig.h b/src/wconfig/WorkerConfig.h index bb4ddf2b0d..bce19b6c86 100644 --- a/src/wconfig/WorkerConfig.h +++ b/src/wconfig/WorkerConfig.h @@ -48,6 +48,7 @@ class AuthContext; // This header declarations namespace lsst::qserv::wconfig { + /// Provide all configuration parameters for a Qserv worker instance. /// Parse an INI configuration file, identify required parameters and ignore /// others, analyze and store them inside private member variables, use default @@ -136,6 +137,9 @@ class WorkerConfig { /// @return slow shared scan priority unsigned int getPrioritySnail() const { return _prioritySnail->getVal(); } + /// @return Prioritize by number of inFLight tasks per scheduler. + bool getPrioritizeByInFlight() const { return _prioritizeByInFlight->getVal(); } + /// @return maximum concurrent chunks for fast shared scan unsigned int getMaxActiveChunksFast() const { return _maxActiveChunksFast->getVal(); } @@ -217,6 +221,13 @@ class WorkerConfig { void setHttpPassword(std::string const& password); http::AuthContext httpAuthContext() const; + /// The number of seconds a czar needs to be incommunicado before being considered + /// dead by a worker. + unsigned int getCzarDeadTimeSec() const { return _czarDeadTimeSec->getVal(); } + + /// Return the number of threads HttpSvc use for communicating with the czar. + unsigned int getCzarComNumHttpThreads() const { return _czarComNumHttpThreads->getVal(); } + /// @return the JSON representation of the configuration parameters. /// @note The object has two collections of the parameters: 'input' - for /// parameters that were proided to the construction of the class, and @@ -287,6 +298,9 @@ class WorkerConfig { util::ConfigValTUInt::create(_configValMap, "scheduler", "priority_med", notReq, 3); CVTUIntPtr _priorityFast = util::ConfigValTUInt::create(_configValMap, "scheduler", "priority_fast", notReq, 4); + CVTBoolPtr _prioritizeByInFlight = + util::ConfigValTBool::create(_configValMap, "results", "prioritize_by_inflight", notReq, false); + CVTUIntPtr _maxReserveSlow = util::ConfigValTUInt::create(_configValMap, "scheduler", "reserve_slow", notReq, 2); CVTUIntPtr _maxReserveSnail = @@ -343,7 +357,7 @@ class WorkerConfig { CVTUIntPtr _replicationHttpPort = util::ConfigValTUInt::create(_configValMap, "replication", "http_port", required, 0); CVTUIntPtr _replicationNumHttpThreads = - util::ConfigValTUInt::create(_configValMap, "replication", "num_http_threads", notReq, 2); + util::ConfigValTUInt::create(_configValMap, "replication", "num_http_threads", notReq, 20); CVTUIntPtr _mysqlPort = util::ConfigValTUInt::create(_configValMap, "mysql", "port", notReq, 4048); CVTStrPtr _mysqlSocket = util::ConfigValTStr::create(_configValMap, "mysql", "socket", notReq, ""); @@ -364,9 +378,13 @@ class WorkerConfig { CVTIntPtr _qPoolMaxPriority = util::ConfigValTInt::create(_configValMap, "qpool", "MaxPriority", notReq, 2); CVTStrPtr _qPoolRunSizes = - util::ConfigValTStr::create(_configValMap, "qpool", "RunSizes", notReq, "30:20:20:10"); + util::ConfigValTStr::create(_configValMap, "qpool", "RunSizes", notReq, "50:20:10"); CVTStrPtr _qPoolMinRunningSizes = - util::ConfigValTStr::create(_configValMap, "qpool", "MinRunningSizes", notReq, "3:3:3:3"); + util::ConfigValTStr::create(_configValMap, "qpool", "MinRunningSizes", notReq, "3:3:3"); + CVTUIntPtr _czarDeadTimeSec = + util::ConfigValTUInt::create(_configValMap, "czar", "DeadTimeSec", notReq, 180); + CVTUIntPtr _czarComNumHttpThreads = + util::ConfigValTUInt::create(_configValMap, "czar", "ComNumHttpThreads", notReq, 40); }; } // namespace lsst::qserv::wconfig diff --git a/src/wcontrol/Foreman.cc b/src/wcontrol/Foreman.cc index a5b90f96e1..f990667359 100644 --- a/src/wcontrol/Foreman.cc +++ b/src/wcontrol/Foreman.cc @@ -50,6 +50,7 @@ #include "wdb/ChunkResource.h" #include "wdb/SQLBackend.h" #include "wpublish/QueriesAndChunks.h" +#include "wsched/BlendScheduler.h" using namespace std; namespace fs = boost::filesystem; @@ -83,15 +84,15 @@ namespace lsst::qserv::wcontrol { Foreman::Ptr Foreman::_globalForeman; -Foreman::Ptr Foreman::create(Scheduler::Ptr const& scheduler, unsigned int poolSize, +Foreman::Ptr Foreman::create(wsched::BlendScheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, mysql::MySqlConfig const& mySqlConfig, wpublish::QueriesAndChunks::Ptr const& queries, - std::shared_ptr const& chunkInventory, - std::shared_ptr const& sqlConnMgr, int qPoolSize, - int maxPriority, std::string const& vectRunSizesStr, - std::string const& vectMinRunningSizesStr) { + shared_ptr const& chunkInventory, + shared_ptr const& sqlConnMgr, int qPoolSize, + int maxPriority, string const& vectRunSizesStr, + string const& vectMinRunningSizesStr) { // Latch - static std::atomic globalForemanSet{false}; + static atomic globalForemanSet{false}; if (globalForemanSet.exchange(true) == true) { throw util::Bug(ERR_LOC, "Foreman::create already an existing global Foreman."); } @@ -102,8 +103,9 @@ Foreman::Ptr Foreman::create(Scheduler::Ptr const& scheduler, unsigned int poolS return _globalForeman; } -Foreman::Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, - mysql::MySqlConfig const& mySqlConfig, wpublish::QueriesAndChunks::Ptr const& queries, +Foreman::Foreman(wsched::BlendScheduler::Ptr const& scheduler, unsigned int poolSize, + unsigned int maxPoolThreads, mysql::MySqlConfig const& mySqlConfig, + wpublish::QueriesAndChunks::Ptr const& queries, std::shared_ptr const& chunkInventory, std::shared_ptr const& sqlConnMgr, int qPoolSize, int maxPriority, std::string const& vectRunSizesStr, std::string const& vectMinRunningSizesStr) @@ -115,7 +117,8 @@ Foreman::Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigne _resourceMonitor(make_shared()), _io_service(), _httpServer(qhttp::Server::create(_io_service, 0 /* grab the first available port */)), - _wCzarInfoMap(WCzarInfoMap::create()) { + _wCzarInfoMap(WCzarInfoMap::create()), + _fqdn(util::getCurrentHostFqdnBlocking()) { // Make the chunk resource mgr // Creating backend makes a connection to the database for making temporary tables. // It will delete temporary tables that it can identify as being created by a worker. @@ -176,10 +179,7 @@ Foreman::~Foreman() { void Foreman::processTasks(vector const& tasks) { std::vector cmds; - for (auto const& task : tasks) { - _queries->addTask(task); - cmds.push_back(task); - } + _queries->addTasks(tasks, cmds); _scheduler->queCmd(cmds); } diff --git a/src/wcontrol/Foreman.h b/src/wcontrol/Foreman.h index ed2f78518a..dd1be58d32 100644 --- a/src/wcontrol/Foreman.h +++ b/src/wcontrol/Foreman.h @@ -72,6 +72,10 @@ class QueryStatistics; // This header declarations +namespace lsst::qserv::wsched { +class BlendScheduler; +} + namespace lsst::qserv::wcontrol { /// An abstract scheduler interface. Foreman objects use Scheduler instances @@ -111,8 +115,8 @@ class Foreman { * @param chunkInventory - a collection of the SSI resources published by the worker * @param sqlConnMgr - for limiting the number of MySQL connections used for tasks */ - static Ptr create(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, - mysql::MySqlConfig const& mySqlConfig, + static Ptr create(std::shared_ptr const& scheduler, unsigned int poolSize, + unsigned int maxPoolThreads, mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& queries, std::shared_ptr const& chunkInventory, std::shared_ptr const& sqlConnMgr, int qPoolSize, int maxPriority, @@ -148,9 +152,15 @@ class Foreman { std::shared_ptr getQueriesAndChunks() const { return _queries; } + std::shared_ptr getScheduler() const { return _scheduler; } + + /// Return the fqdn for this worker. + std::string getFqdn() const { return _fqdn; } + private: - Foreman(Scheduler::Ptr const& scheduler, unsigned int poolSize, unsigned int maxPoolThreads, - mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& queries, + Foreman(std::shared_ptr const& scheduler, unsigned int poolSize, + unsigned int maxPoolThreads, mysql::MySqlConfig const& mySqlConfig, + std::shared_ptr const& queries, std::shared_ptr const& chunkInventory, std::shared_ptr const& sqlConnMgr, int qPoolSize, int maxPriority, std::string const& vectRunSizesStr, std::string const& vectMinRunningSizesStr); @@ -162,7 +172,7 @@ class Foreman { std::shared_ptr _chunkResourceMgr; util::ThreadPool::Ptr _pool; - Scheduler::Ptr _scheduler; + std::shared_ptr _scheduler; util::CommandQueue::Ptr _workerCommandQueue; ///< dedicated queue for the worker commands util::ThreadPool::Ptr _workerCommandPool; ///< dedicated pool for executing worker commands @@ -197,6 +207,9 @@ class Foreman { /// Map of czar information for all czars that have contacted this worker. std::shared_ptr const _wCzarInfoMap; + /// FQDN for this worker. + std::string const _fqdn; + static Ptr _globalForeman; ///< Pointer to the global instance. }; diff --git a/src/wcontrol/SqlConnMgr.cc b/src/wcontrol/SqlConnMgr.cc index 3c2d654969..f5df3ba4fc 100644 --- a/src/wcontrol/SqlConnMgr.cc +++ b/src/wcontrol/SqlConnMgr.cc @@ -52,7 +52,7 @@ SqlConnMgr::ConnType SqlConnMgr::_take(bool scanQuery, shared_ptr const& channelShared, bool firstChannelSqlConn) { ++_totalCount; - LOGS(_log, LOG_LVL_DEBUG, "SqlConnMgr take " << dump()); + LOGS(_log, LOG_LVL_TRACE, "SqlConnMgr take " << dump()); unique_lock uLock(_mtx); SqlConnMgr::ConnType connType = SCAN; @@ -100,7 +100,7 @@ SqlConnMgr::ConnType SqlConnMgr::_take(bool scanQuery, // requestor got its sql connection, increment counts if (channelShared != nullptr) { int newCount = channelShared->incrSqlConnectionCount(); - LOGS(_log, LOG_LVL_DEBUG, "SqlConnMgr::_take newCount=" << newCount); + LOGS(_log, LOG_LVL_TRACE, "SqlConnMgr::_take newCount=" << newCount); } if (connType == SCAN) { @@ -120,7 +120,7 @@ void SqlConnMgr::_release(SqlConnMgr::ConnType connType) { // causing _take() to block when it really should not. // When the FileChannelShared is finished, it is thrown away, effectively // clearing its count. - LOGS(_log, LOG_LVL_DEBUG, "SqlConnMgr release " << dump()); + LOGS(_log, LOG_LVL_TRACE, "SqlConnMgr release " << dump()); if (connType == SCAN) { --_sqlScanConnCount; } else { diff --git a/src/wcontrol/WCzarInfoMap.cc b/src/wcontrol/WCzarInfoMap.cc index 831022c44d..d1554de3f4 100644 --- a/src/wcontrol/WCzarInfoMap.cc +++ b/src/wcontrol/WCzarInfoMap.cc @@ -137,8 +137,8 @@ void WCzarInfo::_sendMessage() { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " Transmit success == 0"); // There's no point in re-sending as the czar got the message and didn't like // it. - // TODO:UJ &&& maybe add this czId+ujId to a list of failed uberjobs that can be put - // TODO:UJ &&& status return??? Probably overkill. + // TODO:UJ Maybe add this czId+ujId to a list of failed uberjobs that can be put + // TODO:UJ status return? } } catch (exception const& ex) { LOGS(_log, LOG_LVL_WARN, cName(__func__) + " " + requestStr + " failed, ex: " + ex.what()); @@ -154,7 +154,8 @@ bool WCzarInfo::checkAlive(TIMEPOINT tmMark) { lock_guard lg(_wciMtx); if (_alive) { auto timeSinceContact = tmMark - _lastTouch; - if (timeSinceContact >= 120s) { // TODO:UJ get _deadTime from config &&& + std::chrono::seconds deadTime(wconfig::WorkerConfig::instance()->getCzarDeadTimeSec()); + if (timeSinceContact >= deadTime) { // Contact with the czar has timed out. LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " czar timeout"); _alive = false; diff --git a/src/wdb/ChunkResource.cc b/src/wdb/ChunkResource.cc index a9fe100e5b..dfefac392b 100644 --- a/src/wdb/ChunkResource.cc +++ b/src/wdb/ChunkResource.cc @@ -84,7 +84,7 @@ std::ostream& operator<<(std::ostream& os, ChunkResource::Info const& i) { ChunkResource::ChunkResource(ChunkResourceMgr* mgr) : _mgr{mgr} {} ChunkResource::ChunkResource(ChunkResourceMgr* mgr, ChunkResource::Info* info) : _mgr{mgr}, _info{info} { - LOGS(_log, LOG_LVL_DEBUG, "ChunkResource info=" << *info); + LOGS(_log, LOG_LVL_TRACE, "ChunkResource info=" << *info); _mgr->acquireUnit(*_info); } ChunkResource::ChunkResource(ChunkResource const& cr) : _mgr{cr._mgr}, _info{new Info(*cr._info)} { @@ -157,8 +157,8 @@ class ChunkEntry { std::lock_guard lock(_mutex); backend->memLockRequireOwnership(); ++_refCount; // Increase usage count - LOGS(_log, LOG_LVL_DEBUG, - "SubChunk acquire refC=" << _refCount << " db=" << db << " tables[" + LOGS(_log, LOG_LVL_TRACE, + "Subchunk acquire refC=" << _refCount << " db=" << db << " tables[" << util::printable(dbTableSet) << "]" << " sc[" << util::printable(sc) << "]"); for (auto const& dbTbl : dbTableSet) { @@ -194,7 +194,7 @@ class ChunkEntry { std::lock_guard lock(_mutex); backend->memLockRequireOwnership(); StringVector::const_iterator ti, te; - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "SubChunk release refC=" << _refCount << " db=" << db << " dbTableSet[" << util::printable(dbTableSet) << "]" << " sc[" << util::printable(sc) << "]"); @@ -275,7 +275,7 @@ ChunkResourceMgr::Ptr ChunkResourceMgr::newMgr(SQLBackend::Ptr const& backend) { ChunkResource ChunkResourceMgr::acquire(std::string const& db, int chunkId, DbTableSet const& tables) { // Make sure that the chunk is ready. (NOP right now.) - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "acquire db=" << db << " chunkId=" << chunkId << " tables=" << util::printable(tables)); ChunkResource cr(this, new ChunkResource::Info(db, chunkId, tables)); return cr; @@ -299,7 +299,7 @@ void ChunkResourceMgr::acquireUnit(ChunkResource::Info const& i) { Map& map = _getMap(i.db); // Select db ChunkEntry& ce = _getChunkEntry(map, i.chunkId); // Actually acquire - LOGS(_log, LOG_LVL_DEBUG, "acquireUnit info=" << i); + LOGS(_log, LOG_LVL_TRACE, "acquireUnit info=" << i); ce.acquire(i.db, i.tables, i.subChunkIds, _backend); } diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index 33b9a287f4..a2e381ffae 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -126,7 +126,8 @@ bool QueryRunner::_initConnection() { bool QueryRunner::runQuery() { util::HoldTrack::Mark runQueryMarkA(ERR_LOC, "runQuery " + to_string(_task->getQueryId())); QSERV_LOGCONTEXT_QUERY_JOB(_task->getQueryId(), _task->getJobId()); - LOGS(_log, LOG_LVL_TRACE, __func__ << " tid=" << _task->getIdStr()); + LOGS(_log, LOG_LVL_TRACE, + "QueryRunner " << _task->cName(__func__) << " scsId=" << _task->getSendChannel()->getScsId()); // Start tracking the task. auto now = chrono::system_clock::now(); @@ -151,34 +152,23 @@ bool QueryRunner::runQuery() { Release release(_task, this, _queriesAndChunks); if (_task->checkCancelled()) { - LOGS(_log, LOG_LVL_DEBUG, "runQuery, task was cancelled before it started." << _task->getIdStr()); + LOGS(_log, LOG_LVL_TRACE, "runQuery, task was cancelled before it started." << _task->getIdStr()); return false; } - if (_task->checkCancelled()) { - LOGS(_log, LOG_LVL_DEBUG, "runQuery, task was cancelled after locking tables."); - return false; - } - - LOGS(_log, LOG_LVL_INFO, - "Exec in flight for Db=" << _task->getDb() << " sqlConnMgr " << _sqlConnMgr->dump()); // Queries that span multiple tasks should not be high priority for the SqlConMgr as it risks deadlock. bool interactive = _task->getScanInteractive() && !(_task->getSendChannel()->getTaskCount() > 1); wcontrol::SqlConnLock sqlConnLock(*_sqlConnMgr, not interactive, _task->getSendChannel()); + bool connOk = _initConnection(); if (!connOk) { // Since there's an error, this will be the last transmit from this QueryRunner. - if (!_task->getSendChannel()->buildAndTransmitError(_multiError, _task, _cancelled)) { - LOGS(_log, LOG_LVL_WARN, " Could not report error to czar as sendChannel not accepting msgs."); - } + _task->getSendChannel()->buildAndTransmitError(_multiError, _task, _cancelled); return false; } // Run the query and send the results back. - if (!_dispatchChannel()) { - return false; - } - return true; + return _dispatchChannel(); } MYSQL_RES* QueryRunner::_primeResult(string const& query) { @@ -238,12 +228,14 @@ bool QueryRunner::_dispatchChannel() { util::Timer primeT; primeT.start(); _task->queryExecutionStarted(); + LOGS(_log, LOG_LVL_TRACE, "QueryRunner " << _task->cName(__func__) << " sql start"); MYSQL_RES* res = _primeResult(query); // This runs the SQL query, throws SqlErrorObj on failure. + LOGS(_log, LOG_LVL_TRACE, "QueryRunner " << _task->cName(__func__) << " sql end"); primeT.stop(); needToFreeRes = true; if (taskSched != nullptr) { taskSched->histTimeOfRunningTasks->addEntry(primeT.getElapsed()); - LOGS(_log, LOG_LVL_DEBUG, "QR " << taskSched->histTimeOfRunningTasks->getString("run")); + LOGS(_log, LOG_LVL_TRACE, "QR " << taskSched->histTimeOfRunningTasks->getString("run")); } else { LOGS(_log, LOG_LVL_ERROR, "QR runtaskSched == nullptr"); } @@ -284,10 +276,7 @@ bool QueryRunner::_dispatchChannel() { erred = true; // Send results. This needs to happen after the error check. // If any errors were found, send an error back. - if (!_task->getSendChannel()->buildAndTransmitError(_multiError, _task, _cancelled)) { - LOGS(_log, LOG_LVL_WARN, - " Could not report error to czar as sendChannel not accepting msgs." << _task->getIdStr()); - } + _task->getSendChannel()->buildAndTransmitError(_multiError, _task, _cancelled); } return !erred; } @@ -295,16 +284,20 @@ bool QueryRunner::_dispatchChannel() { void QueryRunner::cancel() { // QueryRunner::cancel() should only be called by Task::cancel() // to keep the bookkeeping straight. - LOGS(_log, LOG_LVL_WARN, "Trying QueryRunner::cancel() call"); - util::HoldTrack::Mark mark(ERR_LOC, "QR cancel() QID=" + _task->getIdStr()); - _cancelled = true; + LOGS(_log, LOG_LVL_TRACE, "Trying QueryRunner::cancel() call " << _task->getIdStr()); + + bool alreadyCancelled = _cancelled.exchange(true); + if (alreadyCancelled) { + LOGS(_log, LOG_LVL_WARN, "already cancelled" << _task->getIdStr()); + return; + } if (_mysqlConn == nullptr) { - LOGS(_log, LOG_LVL_WARN, "QueryRunner::cancel() no MysqlConn"); + LOGS(_log, LOG_LVL_TRACE, "QueryRunner::cancel() no MysqlConn"); } else { switch (_mysqlConn->cancel()) { case -1: - LOGS(_log, LOG_LVL_WARN, "QueryRunner::cancel() NOP"); + LOGS(_log, LOG_LVL_ERROR, "QueryRunner::cancel() NOP"); break; case 0: LOGS(_log, LOG_LVL_WARN, "QueryRunner::cancel() success"); diff --git a/src/wdb/QueryRunner.h b/src/wdb/QueryRunner.h index bebe22a208..6d5236d95d 100644 --- a/src/wdb/QueryRunner.h +++ b/src/wdb/QueryRunner.h @@ -55,7 +55,8 @@ class QueriesAndChunks; namespace lsst::qserv::wdb { -/// On the worker, run a query related to a Task, writing the results to a table or supplied SendChannel. +/// On the worker, run a query related to a Task, hold the resources needed to run the query, +/// and write the results to the supplied SendChannel. /// class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_from_this { public: diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index c67acf74a2..1bc2f885c6 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -99,8 +99,6 @@ struct Fixture { {"jobId", mInfo.jobId}, {"attemptCount", mInfo.attemptCount}, {"querySpecDb", mInfo.db}, - {"scanPriority", mInfo.scanRating}, - {"scanInteractive", mInfo.scanInteractive}, {"maxTableSize", mInfo.maxTableSize}, {"chunkScanTables", nlohmann::json::array()}, {"chunkId", mInfo.chunkId}, @@ -161,15 +159,17 @@ BOOST_AUTO_TEST_CASE(Simple) { shared_ptr crm = ChunkResourceMgr::newMgr(backend); SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); scanInfo->scanRating = mInfo.scanRating; scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, + mInfo.foreman, mInfo.authKey); + vector taskVect = - Task::createTasksForUnitTest(ujData, *msgJson, sChannel, scanInfo, mInfo.scanInteractive, - mInfo.maxTableSize, crm); + Task::createTasksForUnitTest(ujData, *msgJson, sChannel, mInfo.maxTableSize, crm); Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); @@ -187,15 +187,15 @@ BOOST_AUTO_TEST_CASE(Output) { shared_ptr crm = ChunkResourceMgr::newMgr(backend); SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); scanInfo->scanRating = mInfo.scanRating; scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - vector taskVect = - Task::createTasksForUnitTest(ujData, *msgJson, sc, scanInfo, mInfo.scanInteractive, - mInfo.maxTableSize, crm); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, + mInfo.foreman, mInfo.authKey); + + vector taskVect = Task::createTasksForUnitTest(ujData, *msgJson, sc, mInfo.maxTableSize, crm); Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); diff --git a/src/wpublish/QueriesAndChunks.cc b/src/wpublish/QueriesAndChunks.cc index 1dd21fc791..1410e819d2 100644 --- a/src/wpublish/QueriesAndChunks.cc +++ b/src/wpublish/QueriesAndChunks.cc @@ -119,8 +119,7 @@ void QueriesAndChunks::setBlendScheduler(shared_ptr cons void QueriesAndChunks::setRequiredTasksCompleted(unsigned int value) { _requiredTasksCompleted = value; } -QueryStatistics::Ptr QueriesAndChunks::addQueryId(QueryId qId, CzarIdType czarId) { - unique_lock guardStats(_queryStatsMapMtx); +QueryStatistics::Ptr QueriesAndChunks::_addQueryId(QueryId qId, CzarIdType czarId) { auto itr = _queryStatsMap.find(qId); QueryStatistics::Ptr stats; if (_queryStatsMap.end() == itr) { @@ -132,6 +131,11 @@ QueryStatistics::Ptr QueriesAndChunks::addQueryId(QueryId qId, CzarIdType czarId return stats; } +QueryStatistics::Ptr QueriesAndChunks::addQueryId(QueryId qId, CzarIdType czarId) { + unique_lock guardStats(_queryStatsMapMtx); + return _addQueryId(qId, czarId); +} + /// Add statistics for the Task, creating a QueryStatistics object if needed. void QueriesAndChunks::addTask(wbase::Task::Ptr const& task) { auto qid = task->getQueryId(); @@ -139,6 +143,17 @@ void QueriesAndChunks::addTask(wbase::Task::Ptr const& task) { auto stats = addQueryId(qid, czId); stats->addTask(task); } +void QueriesAndChunks::addTasks(vector const& tasks, + std::vector& cmds) { + unique_lock guardStats(_queryStatsMapMtx); + for (auto const& task : tasks) { + auto qid = task->getQueryId(); + auto czId = task->getCzarId(); + auto stats = _addQueryId(qid, czId); + stats->addTask(task); + cmds.push_back(task); + } +} /// Update statistics for the Task that was just queued. void QueriesAndChunks::queuedTask(wbase::Task::Ptr const& task) { @@ -749,7 +764,7 @@ void ChunkTableStats::addTaskFinished(double minutes) { } else { _data.avgCompletionTime = minutes; } - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "ChkId=" << _chunkId << ":tbl=" << _scanTableName << " completed=" << _data.tasksCompleted << " avgCompletionTime=" << _data.avgCompletionTime); } diff --git a/src/wpublish/QueriesAndChunks.h b/src/wpublish/QueriesAndChunks.h index b89458ba3c..a3d89e1e64 100644 --- a/src/wpublish/QueriesAndChunks.h +++ b/src/wpublish/QueriesAndChunks.h @@ -199,14 +199,11 @@ class QueriesAndChunks { /// @see addQueryId() QueryStatistics::Ptr getStats(QueryId qId) const; - /// Return the statistics for a user query, creating if needed. - /// Since it is possible to get messages out of order, there - /// are several case where something like a cancellation - /// message arrives before any tasks have been created. - /// @see getStats() + /// @see _addQueryId QueryStatistics::Ptr addQueryId(QueryId qId, CzarIdType czarId); void addTask(wbase::Task::Ptr const& task); + void addTasks(std::vector const& tasks, std::vector& cmds); void queuedTask(wbase::Task::Ptr const& task); void startedTask(wbase::Task::Ptr const& task); void finishedTask(wbase::Task::Ptr const& task); @@ -254,6 +251,14 @@ class QueriesAndChunks { static Ptr _globalQueriesAndChunks; QueriesAndChunks(std::chrono::seconds deadAfter, std::chrono::seconds examineAfter); + /// Return the statistics for a user query, creating if needed. + /// Since it is possible to get messages out of order, there + /// are several case where something like a cancellation + /// message arrives before any tasks have been created. + /// @see getStats() + /// _queryStatsMapMtx must be locked before calling. + QueryStatistics::Ptr _addQueryId(QueryId qId, CzarIdType czarId); + /// @return the statistics for a user query. /// _queryStatsMtx must be locked before calling. QueryStatistics::Ptr _getStats(QueryId const& qId) const; diff --git a/src/wsched/BlendScheduler.cc b/src/wsched/BlendScheduler.cc index b5b37346f4..0d1ab2c0a9 100644 --- a/src/wsched/BlendScheduler.cc +++ b/src/wsched/BlendScheduler.cc @@ -95,8 +95,9 @@ BlendScheduler::BlendScheduler(string const& name, wpublish::QueriesAndChunks::P _scanSnail->setDefaultPosition(position++); assert(_schedulers.size() >= 2); // Must have at least _group and _scanSnail in the list. _sortScanSchedulers(); + LOGS(_log, LOG_LVL_INFO, "BlendScheduler _schedMaxThreads=" << _schedMaxThreads); for (auto const& sched : _schedulers) { - LOGS(_log, LOG_LVL_DEBUG, "Scheduler " << _name << " found scheduler " << sched->getName()); + LOGS(_log, LOG_LVL_INFO, "Scheduler " << _name << " found scheduler " << sched->getName()); } } @@ -110,8 +111,8 @@ void BlendScheduler::_sortScanSchedulers() { if (a == _scanSnail) return false; if (b == _scanSnail) return true; - // base on the number of scans in flight. if (_prioritizeByInFlight) { + // More scans in flight means lower priority. auto aInFlight = a->getInFlight() - a->getPriority(); auto bInFlight = b->getInFlight() - b->getPriority(); if (aInFlight < bInFlight) return true; @@ -131,7 +132,15 @@ void BlendScheduler::_sortScanSchedulers() { str += sched->getName() + ", "; } } - LOGS(_log, LOG_LVL_DEBUG, str); + LOGS(_log, LOG_LVL_TRACE, str); +} + +void BlendScheduler::queTaskLoad(util::Command::Ptr const& cmd) { + { + lock_guard guardA(util::CommandQueue::_mx); + _taskLoadQueue.push_back(cmd); + } + notify(false); } void BlendScheduler::queCmd(util::Command::Ptr const& cmd) { @@ -158,9 +167,10 @@ void BlendScheduler::queCmd(std::vector const& cmds) { throw util::Bug(ERR_LOC, "BlendScheduler::queCmd cmds.size() > 1 when no task was set."); } { - util::LockGuardTimed guard(util::CommandQueue::_mx, "BlendScheduler::queCmd a"); + lock_guard guardA(util::CommandQueue::_mx); _ctrlCmdQueue.queCmd(cmd); } + notify(true); // notify all=true continue; } @@ -169,7 +179,7 @@ void BlendScheduler::queCmd(std::vector const& cmds) { QSERV_LOGCONTEXT_QUERY_JOB(task->getQueryId(), task->getJobId()); } - util::LockGuardTimed guard(util::CommandQueue::_mx, "BlendScheduler::queCmd b"); + lock_guard guardB(util::CommandQueue::_mx); // Check for scan tables. The information for all tasks should be the same // as they all belong to the same query, so only examine the first task. if (first) { @@ -179,7 +189,7 @@ void BlendScheduler::queCmd(std::vector const& cmds) { bool interactive = task->getScanInteractive(); if (scanTables.size() <= 0 || interactive) { // If there are no scan tables, no point in putting on a shared scan. - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "Blend chose group scanTables.size=" << scanTables.size() << " interactive=" << interactive); onInteractive = true; @@ -187,13 +197,13 @@ void BlendScheduler::queCmd(std::vector const& cmds) { } else { onInteractive = false; int scanPriority = task->getScanInfo()->scanRating; - if (LOG_CHECK_LVL(_log, LOG_LVL_DEBUG)) { + if (LOG_CHECK_LVL(_log, LOG_LVL_TRACE)) { ostringstream ss; ss << "Blend chose scan for priority=" << scanPriority << " : "; for (auto scanTbl : scanTables) { ss << scanTbl.db + "." + scanTbl.table + " "; } - LOGS(_log, LOG_LVL_DEBUG, ss.str()); + LOGS(_log, LOG_LVL_TRACE, ss.str()); } { // Find the scheduler responsible for this 'scanPriority'. lock_guard lg(_schedMtx); @@ -224,7 +234,7 @@ void BlendScheduler::queCmd(std::vector const& cmds) { task->setTaskScheduler(targSched); _queries->queuedTask(task); taskCmds.push_back(task); - LOGS(_log, LOG_LVL_INFO, + LOGS(_log, LOG_LVL_TRACE, "BlendScheduler::queCmd added tid=" << task->getIdStr() << " sched=" << targSched->getName()); } @@ -232,7 +242,7 @@ void BlendScheduler::queCmd(std::vector const& cmds) { _logSchedulers(); if (!taskCmds.empty()) { - LOGS(_log, LOG_LVL_DEBUG, "Blend queCmd"); + LOGS(_log, LOG_LVL_TRACE, "Blend queCmd"); targSched->queCmd(taskCmds); if (queryStats) { queryStats->tasksAddedToScheduler(targSched, taskCmds.size()); @@ -245,13 +255,14 @@ void BlendScheduler::queCmd(std::vector const& cmds) { void BlendScheduler::commandStart(util::Command::Ptr const& cmd) { auto t = dynamic_pointer_cast(cmd); if (t == nullptr) { - LOGS(_log, LOG_LVL_ERROR, "BlendScheduler::commandStart cmd failed conversion"); + // This happens with loader and control commands. + LOGS(_log, LOG_LVL_TRACE, "BlendScheduler::commandStart cmd not a Task"); return; } QSERV_LOGCONTEXT_QUERY_JOB(t->getQueryId(), t->getJobId()); - LOGS(_log, LOG_LVL_DEBUG, "BlendScheduler::commandStart"); + LOGS(_log, LOG_LVL_TRACE, "BlendScheduler::commandStart"); wcontrol::Scheduler::Ptr s = dynamic_pointer_cast(t->getTaskScheduler()); if (s != nullptr) { s->commandStart(t); @@ -259,28 +270,29 @@ void BlendScheduler::commandStart(util::Command::Ptr const& cmd) { LOGS(_log, LOG_LVL_ERROR, "BlendScheduler::commandStart scheduler not found"); } _infoChanged = true; - LOGS(_log, LOG_LVL_DEBUG, "BlendScheduler::commandStart &&& end"); } void BlendScheduler::commandFinish(util::Command::Ptr const& cmd) { auto t = dynamic_pointer_cast(cmd); if (t == nullptr) { - LOGS(_log, LOG_LVL_WARN, "BlendScheduler::commandFinish cmd failed conversion"); + LOGS(_log, LOG_LVL_TRACE, "BlendScheduler::commandFinish cmd is not a Task"); return; } QSERV_LOGCONTEXT_QUERY_JOB(t->getQueryId(), t->getJobId()); wcontrol::Scheduler::Ptr s = dynamic_pointer_cast(t->getTaskScheduler()); - LOGS(_log, LOG_LVL_DEBUG, "BlendScheduler::commandFinish"); + LOGS(_log, LOG_LVL_TRACE, "BlendScheduler::commandFinish"); if (s != nullptr) { s->commandFinish(t); } else { LOGS(_log, LOG_LVL_ERROR, "BlendScheduler::commandFinish scheduler not found"); } _infoChanged = true; - _logChunkStatus(); - notify(true); // notify all=true + if (LOG_CHECK_LVL(_log, LOG_LVL_TRACE)) { + _logChunkStatus(); + } + notify(false); // notify one } bool BlendScheduler::ready() { @@ -301,22 +313,27 @@ bool BlendScheduler::_ready() { ostringstream os; bool ready = false; + if (_taskLoadQueue.size() > 0) { + ready = true; + return ready; + } + // _readSched points to the scheduler with a ready task until that // task has been retrieved by getCmd(). if (_readySched != nullptr) { ready = true; } - - // Get the total number of threads schedulers want reserved - int availableThreads = calcAvailableTheads(); bool changed = _infoChanged.exchange(false); if (!ready) { lock_guard lg(_schedMtx); + + // Get the total number of threads schedulers want reserved + int availableThreads = _calcAvailableTheads(); for (auto const& sched : _schedulers) { availableThreads = sched->applyAvailableThreads(availableThreads); ready = sched->ready(); - if (changed && LOG_CHECK_LVL(_log, LOG_LVL_DEBUG)) { + if (changed && LOG_CHECK_LVL(_log, LOG_LVL_TRACE)) { os << sched->getName() << "(r=" << ready << " sz=" << sched->getSize() << " fl=" << sched->getInFlight() << " avail=" << availableThreads << ") "; } @@ -327,73 +344,70 @@ bool BlendScheduler::_ready() { } } - // IF nothing ready on the schedulers, check if the thread pool size should be changed. + // If nothing ready on the schedulers, check if the thread pool size should be changed. if (!ready) { ready = _ctrlCmdQueue.ready(); } if (changed) { - LOGS(_log, LOG_LVL_DEBUG, getName() << "_ready() " << os.str()); + LOGS(_log, LOG_LVL_TRACE, getName() << "_ready() " << os.str()); } return ready; } +atomic logChunkLimiter = 0; + util::Command::Ptr BlendScheduler::getCmd(bool wait) { - util::Timer timeToLock; - util::Timer timeHeld; util::Command::Ptr cmd; - double totalTimeHeld = 0.0; bool ready = false; { - timeToLock.start(); unique_lock lock(util::CommandQueue::_mx); - timeToLock.stop(); - timeHeld.start(); if (wait) { - // util::CommandQueue::_cv.wait(lock, [this](){return _ready();}); - while (!_ready()) { - timeHeld.stop(); - totalTimeHeld += timeHeld.getElapsed(); - util::CommandQueue::_cv.wait(lock); - timeHeld.start(); - } + util::CommandQueue::_cv.wait(lock, [this]() { return _ready(); }); ready = true; } else { ready = _ready(); } + if (ready && _taskLoadQueue.size() > 0) { + cmd = _taskLoadQueue.front(); + _taskLoadQueue.pop_front(); + notify(false); + return cmd; + } + _logSchedulers(); // Try to get a command from the schedulers if (ready && (_readySched != nullptr)) { cmd = _readySched->getCmd(false); if (cmd != nullptr) { - wbase::Task::Ptr task = dynamic_pointer_cast(cmd); - LOGS(_log, LOG_LVL_DEBUG, - "Blend getCmd() using cmd from " << _readySched->getName() << " chunkId=" - << task->getChunkId() << " QID=" << task->getIdStr()); + _sortScanSchedulers(); + if (LOG_CHECK_LVL(_log, LOG_LVL_TRACE)) { + wbase::Task::Ptr task = dynamic_pointer_cast(cmd); + LOGS(_log, LOG_LVL_TRACE, + "Blend getCmd() using cmd from " << _readySched->getName() + << " chunkId=" << task->getChunkId() + << " QID=" << task->getIdStr()); + } } _readySched.reset(); - _sortScanSchedulers(); } + } - if (cmd == nullptr) { - // The scheduler didn't have anything, see if there's anything on the control queue, - // which could change the size of the pool. - cmd = _ctrlCmdQueue.getCmd(); - } + if (cmd == nullptr) { + // The scheduler didn't have anything, see if there's anything on the control queue, + // which could change the size of the pool. + cmd = _ctrlCmdQueue.getCmd(); } + if (cmd != nullptr) { _infoChanged = true; - _logChunkStatus(); + if (LOG_CHECK_LVL(_log, LOG_LVL_TRACE) || (logChunkLimiter++ % 100 == 0)) { + _logChunkStatus(); + } notify(false); // notify all=false } // returning nullptr is acceptable. - timeHeld.stop(); - totalTimeHeld += timeHeld.getElapsed(); - LOGS(_log, LOG_LVL_DEBUG, - "lockTime BlendScheduler::getCmd ready toLock=" << timeToLock.getElapsed() - << " held=" << timeHeld.getElapsed() - << " totalHeld=" << totalTimeHeld); return cmd; } @@ -408,17 +422,16 @@ int BlendScheduler::_getAdjustedMaxThreads(int oldAdjMax, int inFlight) { } /// @return the number of threads that are not reserved by any sub-scheduler. -int BlendScheduler::calcAvailableTheads() { +int BlendScheduler::_calcAvailableTheads() { int reserve = 0; { - lock_guard lg(_schedMtx); for (auto const& sched : _schedulers) { reserve += sched->desiredThreadReserve(); } } int available = _schedMaxThreads - reserve; if (available < 0) { - LOGS(_log, LOG_LVL_DEBUG, "calcAvailableTheads negative available=" << available); + LOGS(_log, LOG_LVL_TRACE, "calcAvailableTheads negative available=" << available); } return available; } diff --git a/src/wsched/BlendScheduler.h b/src/wsched/BlendScheduler.h index 1ee2b65956..e25eba7bd6 100644 --- a/src/wsched/BlendScheduler.h +++ b/src/wsched/BlendScheduler.h @@ -105,6 +105,8 @@ class BlendScheduler : public wsched::SchedulerBase { BlendScheduler& operator=(BlendScheduler const&) = delete; ~BlendScheduler() override = default; + void queTaskLoad(util::Command::Ptr const& cmd); + void queCmd(util::Command::Ptr const& cmd) override; void queCmd(std::vector const& cmds) override; util::Command::Ptr getCmd(bool wait) override; @@ -118,8 +120,6 @@ class BlendScheduler : public wsched::SchedulerBase { bool ready() override; int applyAvailableThreads(int tempMax) override { return tempMax; } //< does nothing - int calcAvailableTheads(); - bool isScanSnail(SchedulerBase::Ptr const& scan); int moveUserQueryToSnail(QueryId qId, SchedulerBase::Ptr const& source); int moveUserQuery(QueryId qId, SchedulerBase::Ptr const& source, SchedulerBase::Ptr const& destination); @@ -138,7 +138,11 @@ class BlendScheduler : public wsched::SchedulerBase { void _sortScanSchedulers(); void _logChunkStatus(); void _logSchedulers(); + + /// _schedMtx must be locked before calling. + int _calcAvailableTheads(); ControlCommandQueue _ctrlCmdQueue; ///< Needed for changing thread pool size. + std::deque _taskLoadQueue; int _schedMaxThreads; ///< maximum number of threads that can run. @@ -152,8 +156,8 @@ class BlendScheduler : public wsched::SchedulerBase { wpublish::QueriesAndChunks::Ptr _queries; /// UserQuery statistics. - std::atomic _prioritizeByInFlight{ - false}; // Schedulers with more tasks inflight get lower priority. + /// Schedulers with more tasks inflight get lower priority. + std::atomic _prioritizeByInFlight{false}; SchedulerBase::Ptr _readySched; //< Pointer to the scheduler with a ready task. /// Record performance data when this value is less than now(), and then this value us increased diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index ff1a19752d..cafbfb8dc2 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -54,7 +54,7 @@ void ChunkTasksQueue::queueTask(std::vector const& tasks) { // Correct ChunkTask wasn't found, make a new one. std::pair ele(chunkId, std::make_shared(chunkId)); auto res = _chunkMap.insert(ele); // insert should fail if the key already exists. - LOGS(_log, LOG_LVL_DEBUG, " queueTask chunk=" << chunkId << " created=" << res.second); + LOGS(_log, LOG_LVL_TRACE, " queueTask chunk=" << chunkId << " created=" << res.second); iter = res.first; } } @@ -95,7 +95,7 @@ bool ChunkTasksQueue::_ready(bool useFlexibleLock) { // If the _activeChunk is invalid, start at the beginning. if (_activeChunk == _chunkMap.end()) { - LOGS(_log, LOG_LVL_INFO, "ChunkTasksQueue::_ready _activeChunk invalid, reset"); + LOGS(_log, LOG_LVL_DEBUG, "ChunkTasksQueue::_ready _activeChunk invalid, reset"); _activeChunk = _chunkMap.begin(); _activeChunk->second->setActive(); // Flag tasks on active so new Tasks added wont be run. } @@ -108,7 +108,7 @@ bool ChunkTasksQueue::_ready(bool useFlexibleLock) { // Should the active chunk be advanced? if (_activeChunk->second->readyToAdvance()) { - LOGS(_log, LOG_LVL_DEBUG, "ChunkTasksQueue::_ready advancing chunk"); + LOGS(_log, LOG_LVL_TRACE, "ChunkTasksQueue::_ready advancing chunk"); auto newActive = _activeChunk; ++newActive; if (newActive == _chunkMap.end()) { @@ -137,7 +137,7 @@ bool ChunkTasksQueue::_ready(bool useFlexibleLock) { // Advance through chunks until READY found, or until entire list scanned. auto iter = _activeChunk; ChunkTasks::ReadyState chunkState = iter->second->ready(useFlexibleLock); - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "_ready loopA state=" << ChunkTasks::toStr(chunkState) << " iter=" << iter->first << " " << iter->second->cInfo()); while (chunkState != ChunkTasks::ReadyState::READY) { @@ -158,7 +158,7 @@ bool ChunkTasksQueue::_ready(bool useFlexibleLock) { } chunkState = iter->second->ready(useFlexibleLock); } - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "_ready loopB state=" << ChunkTasks::toStr(chunkState) << " iter=" << iter->first << " " << iter->second->cInfo()); _readyChunk = iter->second; @@ -306,21 +306,21 @@ void ChunkTasks::queTask(wbase::Task::Ptr const& a) { _activeTasks.push(a); state = "ACTIVE"; } - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "ChunkTasks::queTask tid=" << a->getIdStr() << " chunkId=" << _chunkId << " state=" << state << " active.sz=" << _activeTasks._tasks.size() << " pend.sz=" << _pendingTasks.size() << cInfo()); if (_activeTasks.empty()) { - LOGS(_log, LOG_LVL_DEBUG, "Top of ACTIVE is now: (empty)"); + LOGS(_log, LOG_LVL_TRACE, "Top of ACTIVE is now: (empty)"); } else { - LOGS(_log, LOG_LVL_DEBUG, "Top of ACTIVE is now: " << _activeTasks.top()->getIdStr()); + LOGS(_log, LOG_LVL_TRACE, "Top of ACTIVE is now: " << _activeTasks.top()->getIdStr()); } } /// Set this chunk as the active chunk and move pending jobs to active if needed. void ChunkTasks::setActive(bool active) { if (_active != active) { - LOGS(_log, LOG_LVL_DEBUG, "ChunkTasks " << _chunkId << " active changed to " << active); + LOGS(_log, LOG_LVL_TRACE, "ChunkTasks " << _chunkId << " active changed to " << active); if (_active && !active) { movePendingToActive(); } @@ -331,7 +331,7 @@ void ChunkTasks::setActive(bool active) { /// Move all pending Tasks to the active heap. void ChunkTasks::movePendingToActive() { for (auto const& t : _pendingTasks) { - LOGS(_log, LOG_LVL_DEBUG, "ChunkTasks " << _chunkId << " pending->active " << t->getIdStr()); + LOGS(_log, LOG_LVL_TRACE, "ChunkTasks " << _chunkId << " pending->active " << t->getIdStr()); _activeTasks.push(t); } _pendingTasks.clear(); @@ -344,8 +344,7 @@ bool ChunkTasks::empty() const { return _activeTasks.empty() && _pendingTasks.em bool ChunkTasks::readyToAdvance() { // There is a rare case where _activeTasks and _inFlightTasks are empty but _readyTask in not null. bool advance = _activeTasks.empty() && _inFlightTasks.empty() && _readyTask == nullptr; - auto logLvl = (advance) ? LOG_LVL_INFO : LOG_LVL_TRACE; - LOGS(_log, logLvl, + LOGS(_log, LOG_LVL_TRACE, "ChunkTasks::readyToAdvance chunkId=" << _chunkId << " _activeTasks.sz=" << _activeTasks.size() << " _inFlightTasks.sz=" << _inFlightTasks.size() << " _readyTask==null=" << (_readyTask == nullptr) @@ -400,19 +399,27 @@ wbase::Task::Ptr ChunkTasks::getTask(bool useFlexibleLock) { void ChunkTasks::taskComplete(wbase::Task::Ptr const& task) { _inFlightTasks.erase(task.get()); } -std::string ChunkTasks::cInfo() const { +std::string ChunkTasks::cInfo(bool listTasks) const { std::stringstream os; os << " cInfo(chkId=" << _chunkId << " act=" << _active << " readyTask=" << _readyTask << " inF=" << _inFlightTasks.size() << " (act=" << _activeTasks.size() << " "; - for (auto const& tsk : _activeTasks._tasks) { - os << tsk->getIdStr() << ", "; + if (listTasks) { + for (auto const& tsk : _activeTasks._tasks) { + os << tsk->getIdStr() << ", "; + } + } else { + os << "..."; } + os << ") (pend.sz=" << _pendingTasks.size() << " "; - for (auto const& tsk : _pendingTasks) { - os << tsk->getIdStr() << ", "; + if (listTasks) { + for (auto const& tsk : _pendingTasks) { + os << tsk->getIdStr() << ", "; + } + } else { + os << "..."; } os << "))"; - return os.str(); } diff --git a/src/wsched/ChunkTasksQueue.h b/src/wsched/ChunkTasksQueue.h index baa48af756..a29c57e1fb 100644 --- a/src/wsched/ChunkTasksQueue.h +++ b/src/wsched/ChunkTasksQueue.h @@ -71,7 +71,7 @@ class ChunkTasks { wbase::Task::Ptr removeTask(wbase::Task::Ptr const& task); /// @return a string describing this instance for the log file. - std::string cInfo() const; + std::string cInfo(bool listTasks = false) const; /// Class that keeps the slowest tables at the front of the heap. class SlowTableHeap { diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index 8695f78bb2..304653b9bf 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -69,14 +69,17 @@ ScanScheduler::ScanScheduler(string const& name, int maxThreads, int maxReserve, } void ScanScheduler::commandStart(util::Command::Ptr const& cmd) { - wbase::Task::Ptr task = dynamic_pointer_cast(cmd); _infoChanged = true; - if (task == nullptr) { - LOGS(_log, LOG_LVL_WARN, "ScanScheduler::commandStart cmd failed conversion " << getName()); - return; + auto logLvl = LOG_LVL_TRACE; + if (LOG_CHECK_LVL(_log, logLvl)) { + wbase::Task::Ptr task = dynamic_pointer_cast(cmd); + if (task == nullptr) { + LOGS(_log, LOG_LVL_WARN, "ScanScheduler::commandStart cmd failed conversion " << getName()); + return; + } + QSERV_LOGCONTEXT_QUERY_JOB(task->getQueryId(), task->getJobId()); + LOGS(_log, logLvl, "commandStart " << getName() << " task=" << task->getIdStr()); } - QSERV_LOGCONTEXT_QUERY_JOB(task->getQueryId(), task->getJobId()); - LOGS(_log, LOG_LVL_DEBUG, "commandStart " << getName() << " task=" << task->getIdStr()); // task was registered Inflight when getCmd() was called. } @@ -96,11 +99,11 @@ void ScanScheduler::commandFinish(util::Command::Ptr const& cmd) { lock_guard guard(util::CommandQueue::_mx); --_inFlight; ++_recentlyCompleted; - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "commandFinish " << getName() << " inFlight=" << _inFlight << " " << task->getIdStr()); _decrChunkTaskCount(task->getChunkId()); } - LOGS(_log, LOG_LVL_DEBUG, "tskEnd chunk=" << task->getChunkId() << " " << task->getIdStr()); + LOGS(_log, LOG_LVL_TRACE, "tskEnd chunk=" << task->getChunkId() << " " << task->getIdStr()); // Whenever a Task finishes, sleeping threads need to check if resources // are available to run new Tasks. _cv.notify_one(); @@ -119,7 +122,7 @@ bool ScanScheduler::_ready() { if (_infoChanged) { _infoChanged = false; logStuff = true; - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, getName() << " ScanScheduler::_ready " << " inFlight=" << _inFlight << " maxThreads=" << _maxThreads << " adj=" << _maxThreadsAdj << " activeChunks=" << getActiveChunkCount() @@ -127,7 +130,7 @@ bool ScanScheduler::_ready() { } if (_inFlight >= maxInFlight()) { if (logStuff) { - LOGS(_log, LOG_LVL_DEBUG, getName() << " ScanScheduler::_ready too many in flight " << _inFlight); + LOGS(_log, LOG_LVL_TRACE, getName() << " ScanScheduler::_ready too many in flight " << _inFlight); } return false; } @@ -156,7 +159,7 @@ util::Command::Ptr ScanScheduler::getCmd(bool wait) { if (task != nullptr) { ++_inFlight; // in flight as soon as it is off the queue. QSERV_LOGCONTEXT_QUERY_JOB(task->getQueryId(), task->getJobId()); - LOGS(_log, LOG_LVL_DEBUG, + LOGS(_log, LOG_LVL_TRACE, "getCmd " << getName() << " tskStart chunk=" << task->getChunkId() << " tid=" << task->getIdStr() << " inflight=" << _inFlight << _taskQueue->queueInfo()); _infoChanged = true; @@ -205,15 +208,13 @@ void ScanScheduler::queCmd(vector const& cmds) { tsk->setMemMan(_memMan); tasks.push_back(tsk); - LOGS(_log, LOG_LVL_INFO, getName() << " queCmd " << tsk->getIdStr()); + LOGS(_log, LOG_LVL_TRACE, getName() << " queCmd " << tsk->getIdStr()); } // Queue the tasks { lock_guard lock(util::CommandQueue::_mx); auto uqCount = _incrCountForUserQuery(qid, tasks.size()); - LOGS(_log, LOG_LVL_DEBUG, - getName() << " queCmd " - << " uqCount=" << uqCount); + LOGS(_log, LOG_LVL_TRACE, getName() << " queCmd " << " uqCount=" << uqCount); _taskQueue->queueTask(tasks); _infoChanged = true; } diff --git a/src/wsched/SchedulerBase.cc b/src/wsched/SchedulerBase.cc index c3981f64d5..c755a0a973 100644 --- a/src/wsched/SchedulerBase.cc +++ b/src/wsched/SchedulerBase.cc @@ -39,6 +39,32 @@ using namespace std; namespace lsst::qserv::wsched { +SchedulerBase::SchedulerBase(std::string const& name, int maxThreads, int maxReserve, int maxActiveChunks, + int priority) + : _name{name}, + _maxReserve{maxReserve}, + _maxReserveDefault{maxReserve}, + _maxThreads{maxThreads}, + _maxThreadsAdj{maxThreads}, + _priority{priority}, + _priorityDefault{priority} { + setMaxActiveChunks(maxActiveChunks); + + using namespace std::chrono_literals; + std::vector bucketMaxVals{0.01, 0.1, 1}; + size_t maxSize = 10; + _histQueuedTasks = std::make_shared("queuedTasks", bucketMaxVals, 1h, maxSize); + _histRunningTasks = std::make_shared("runningTasks", bucketMaxVals, 1h, maxSize); + _histTransmittingTasks = + std::make_shared("transmittingTasks", bucketMaxVals, 1h, maxSize); + _histRecentlyCompletedTasks = + std::make_shared("recentlyCompletedTasks", bucketMaxVals, 1h, maxSize); + + LOGS(_log, LOG_LVL_INFO, + "Scheduler name=" << name << " maxThreads=" << _maxThreads << " maxThreads=" << _maxThreads + << " priority=" << _priority); +} + /// Set priority to use when starting next chunk. void SchedulerBase::setPriority(int priority) { _priority = priority; } @@ -61,7 +87,7 @@ int SchedulerBase::_decrCountForUserQuery(QueryId queryId) { count = --(iter->second); if (count <= 0) { _userQueryCounts.erase(iter); - LOGS(_log, LOG_LVL_DEBUG, queryId << " uqCount=0, erased"); + LOGS(_log, LOG_LVL_TRACE, queryId << " uqCount=0, erased"); } } return count; diff --git a/src/wsched/SchedulerBase.h b/src/wsched/SchedulerBase.h index 7f6e9047b3..36e63023e8 100644 --- a/src/wsched/SchedulerBase.h +++ b/src/wsched/SchedulerBase.h @@ -48,29 +48,7 @@ class SchedulerBase : public wcontrol::Scheduler { static int getMaxPriority() { return 1000000000; } - SchedulerBase(std::string const& name, int maxThreads, int maxReserve, int maxActiveChunks, int priority) - : _name{name}, - _maxReserve{maxReserve}, - _maxReserveDefault{maxReserve}, - _maxThreads{maxThreads}, - _maxThreadsAdj{maxThreads}, - _priority{priority}, - _priorityDefault{priority} { - setMaxActiveChunks(maxActiveChunks); - - using namespace std::chrono_literals; - // TODO: DM-??? set values from configuration, change values at runtime. - std::vector bucketMaxVals{0.01, 0.1, 1}; - size_t maxSize = 10; - _histQueuedTasks = - std::make_shared("queuedTasks", bucketMaxVals, 1h, maxSize); - _histRunningTasks = - std::make_shared("runningTasks", bucketMaxVals, 1h, maxSize); - _histTransmittingTasks = - std::make_shared("transmittingTasks", bucketMaxVals, 1h, maxSize); - _histRecentlyCompletedTasks = std::make_shared("recentlyCompletedTasks", - bucketMaxVals, 1h, maxSize); - } + SchedulerBase(std::string const& name, int maxThreads, int maxReserve, int maxActiveChunks, int priority); virtual ~SchedulerBase() {} SchedulerBase(SchedulerBase const&) = delete; SchedulerBase& operator=(SchedulerBase const&) = delete; diff --git a/src/xrdsvc/HttpModule.cc b/src/xrdsvc/HttpModule.cc index 02f46818d9..06c448200f 100644 --- a/src/xrdsvc/HttpModule.cc +++ b/src/xrdsvc/HttpModule.cc @@ -88,10 +88,9 @@ wbase::TaskSelector HttpModule::translateTaskSelector(string const& func) const } } selector.maxTasks = query().optionalUInt("max_tasks", 0); - debug(func, "include_tasks=" + string(selector.includeTasks ? "1" : "0")); - debug(func, "queryIds.size()=" + to_string(selector.queryIds.size())); - debug(func, "taskStates.size()=" + to_string(selector.taskStates.size())); - debug(func, "max_tasks=" + to_string(selector.maxTasks)); + trace(func, "include_tasks=" + string(selector.includeTasks ? "1" : "0") + + " queryIds.size()=" + to_string(selector.queryIds.size()) + " taskStates.size()=" + + to_string(selector.taskStates.size()) + " max_tasks=" + to_string(selector.maxTasks)); return selector; } diff --git a/src/xrdsvc/HttpSvc.cc b/src/xrdsvc/HttpSvc.cc index 0908efcaaa..6d1df4d0d0 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/xrdsvc/HttpSvc.cc @@ -155,7 +155,8 @@ uint16_t HttpSvc::start() { _threads.push_back(make_unique([self]() { self->_io_service.run(); })); } auto const actualPort = _httpServerPtr->getPort(); - LOGS(_log, LOG_LVL_INFO, context + "started on port " + to_string(actualPort)); + LOGS(_log, LOG_LVL_INFO, + context + "started on port " + to_string(actualPort) + " numThreads=" + to_string(_numThreads)); return actualPort; } diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 8a4aa910b1..b9446ec62d 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -40,6 +40,9 @@ #include "protojson/UberJobMsg.h" #include "protojson/WorkerQueryStatusData.h" #include "qmeta/types.h" +#include "util/Command.h" +#include "util/Error.h" +#include "util/MultiError.h" #include "util/String.h" #include "util/Timer.h" #include "wbase/FileChannelShared.h" @@ -53,6 +56,7 @@ #include "wpublish/ChunkInventory.h" #include "wpublish/QueriesAndChunks.h" #include "wpublish/QueryStatistics.h" +#include "wsched/BlendScheduler.h" #include "xrdsvc/SsiProvider.h" #include "xrdsvc/XrdName.h" @@ -108,17 +112,21 @@ json HttpWorkerCzarModule::_queryJob() { json HttpWorkerCzarModule::_handleQueryJob(string const& func) { json jsRet; - vector ujTasks; + try { auto const& jsReq = body().objJson; auto uberJobMsg = protojson::UberJobMsg::createFromJson(jsReq); UberJobId ujId = uberJobMsg->getUberJobId(); auto ujCzInfo = uberJobMsg->getCzarContactInfo(); - auto czarId = ujCzInfo->czId; QueryId ujQueryId = uberJobMsg->getQueryId(); int ujRowLimit = uberJobMsg->getRowLimit(); auto targetWorkerId = uberJobMsg->getWorkerId(); + uint64_t maxTableSizeMb = uberJobMsg->getMaxTableSizeMb(); + uint64_t const MB_SIZE_BYTES = 1024 * 1024; + uint64_t maxTableSizeBytes = maxTableSizeMb * MB_SIZE_BYTES; + auto scanInfo = uberJobMsg->getScanInfo(); + bool scanInteractive = uberJobMsg->getScanInteractive(); // Get or create QueryStatistics and UserQueryInfo instances. auto queryStats = foreman()->getQueriesAndChunks()->addQueryId(ujQueryId, ujCzInfo->czId); @@ -133,9 +141,48 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { to_string(ujQueryId) + " ujId=" + to_string(ujId)); } + std::shared_ptr foremanPtr = foreman(); + std::string authKeyStr = authKey(); + + // It is important to create UberJobData at this point as it will be the only way to + // inform the czar of errors after this function returns. auto ujData = wbase::UberJobData::create(ujId, ujCzInfo->czName, ujCzInfo->czId, ujCzInfo->czHostName, - ujCzInfo->czPort, ujQueryId, ujRowLimit, targetWorkerId, - foreman(), authKey()); + ujCzInfo->czPort, ujQueryId, ujRowLimit, maxTableSizeBytes, + scanInfo, scanInteractive, targetWorkerId, foremanPtr, + authKeyStr, foremanPtr->httpPort()); + + auto lFunc = [ujId, ujQueryId, ujCzInfo, ujRowLimit, maxTableSizeBytes, targetWorkerId, userQueryInfo, + uberJobMsg, foremanPtr, authKeyStr, ujData](util::CmdData*) { + _buildTasks(ujId, ujQueryId, ujCzInfo, ujRowLimit, maxTableSizeBytes, targetWorkerId, + userQueryInfo, uberJobMsg, foremanPtr, authKeyStr, ujData); + }; + + util::Command::Ptr taskLoadCmd = std::make_shared(lFunc); + foremanPtr->getScheduler()->queTaskLoad(taskLoadCmd); + + string note = string("qId=") + to_string(ujQueryId) + " ujId=" + to_string(ujId); + jsRet = {{"success", 1}, {"errortype", "none"}, {"note", note}}; + LOGS(_log, LOG_LVL_TRACE, "_handleQueryJob jsRet=" << jsRet); + } catch (wbase::TaskException const& texp) { + LOGS(_log, LOG_LVL_ERROR, + "HttpWorkerCzarModule::_handleQueryJob wbase::TaskException received " << texp.what()); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", texp.what()}}; + } + return jsRet; +} + +void HttpWorkerCzarModule::_buildTasks(UberJobId ujId, QueryId ujQueryId, + protojson::CzarContactInfo::Ptr const& ujCzInfo, int ujRowLimit, + uint64_t maxTableSizeBytes, string const& targetWorkerId, + std::shared_ptr const& userQueryInfo, + protojson::UberJobMsg::Ptr const& uberJobMsg, + shared_ptr const& foremanPtr, + string const& authKeyStr, wbase::UberJobData::Ptr const& ujData) { + try { + LOGS(_log, LOG_LVL_TRACE, __func__ << " qid=" << ujQueryId << "ujId=" << ujId); + util::Timer timerParse; + timerParse.start(); + auto czarId = ujCzInfo->czId; // Find the entry for this queryId, create a new one if needed. userQueryInfo->addUberJob(ujData); @@ -145,33 +192,34 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { ujData->setFileChannelShared(channelShared); auto ujTasks = wbase::Task::createTasksFromUberJobMsg( - uberJobMsg, ujData, channelShared, foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), - foreman()->sqlConnMgr(), foreman()->queriesAndChunks(), foreman()->httpPort()); + uberJobMsg, ujData, channelShared, foremanPtr->chunkResourceMgr(), foremanPtr->mySqlConfig(), + foremanPtr->sqlConnMgr(), foremanPtr->queriesAndChunks()); channelShared->setTaskCount(ujTasks.size()); ujData->addTasks(ujTasks); - // At this point, it looks like the message was sent successfully, update - // czar touched time. - wcontrol::WCzarInfoMap::Ptr wCzarMap = foreman()->getWCzarInfoMap(); + // At this point, it looks like the message was sent successfully. + wcontrol::WCzarInfoMap::Ptr wCzarMap = foremanPtr->getWCzarInfoMap(); wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czarId); wCzarInfo->czarMsgReceived(CLOCK::now()); + timerParse.stop(); util::Timer timer; timer.start(); - foreman()->processTasks(ujTasks); // Queues tasks to be run later. + foremanPtr->processTasks(ujTasks); // Queues tasks to be run later. timer.stop(); - LOGS(_log, LOG_LVL_DEBUG, - __func__ << " Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); - - string note = string("qId=") + to_string(ujQueryId) + " ujId=" + to_string(ujId) + - " tasks in uberJob=" + to_string(channelShared->getTaskCount()); - jsRet = {{"success", 1}, {"errortype", "none"}, {"note", note}}; + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " Enqueued UberJob time=" << timer.getElapsed() + << " parseTime=" << timerParse.getElapsed() << " " << uberJobMsg->getIdStr()); } catch (wbase::TaskException const& texp) { - LOGS(_log, LOG_LVL_ERROR, "wbase::TaskException received " << texp.what()); - jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", texp.what()}}; + LOGS(_log, LOG_LVL_ERROR, + "HttpWorkerCzarModule::_buildTasks wbase::TaskException received " << texp.what()); + // Send a message back saying this UberJobFailed + util::MultiError multiErr; + util::Error err(-1, string("UberJob parse error ") + texp.what()); + multiErr.push_back(err); + ujData->responseError(multiErr, -1, false, LOG_LVL_ERROR); } - return jsRet; } json HttpWorkerCzarModule::_queryStatus() { diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/xrdsvc/HttpWorkerCzarModule.h index 500c905e5c..00a6d4fd67 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.h +++ b/src/xrdsvc/HttpWorkerCzarModule.h @@ -34,11 +34,21 @@ #include "qmeta/types.h" #include "xrdsvc/HttpModule.h" +namespace lsst::qserv::protojson { +class CzarContactInfo; +class UberJobMsg; +} // namespace lsst::qserv::protojson + namespace lsst::qserv::qhttp { class Request; class Response; } // namespace lsst::qserv::qhttp +namespace lsst::qserv::wbase { +class UberJobData; +class UserQueryInfo; +} // namespace lsst::qserv::wbase + namespace lsst::qserv::wcontrol { class Foreman; } // namespace lsst::qserv::wcontrol @@ -82,6 +92,14 @@ class HttpWorkerCzarModule : public xrdsvc::HttpModule { /// work of deciphering the message, creating UberJobData objects and Task objects. nlohmann::json _handleQueryJob(std::string const& func); + static void _buildTasks(UberJobId ujId, QueryId ujQueryId, + std::shared_ptr const& ujCzInfo, int ujRowLimit, + uint64_t maxTableSizeBytes, std::string const& targetWorkerId, + std::shared_ptr const& userQueryInfo, + std::shared_ptr const& uberJobMsg, + std::shared_ptr const& foremanPtr, + std::string const& authKeyStr, std::shared_ptr const& ujData); + /// Verify some aspects of the query and call _handleQueryStatus nlohmann::json _queryStatus(); diff --git a/src/xrdsvc/SsiService.cc b/src/xrdsvc/SsiService.cc new file mode 100644 index 0000000000..40c85c4a8f --- /dev/null +++ b/src/xrdsvc/SsiService.cc @@ -0,0 +1,291 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2015-2016 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "xrdsvc/SsiService.h" + +// System headers +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +// Third-party headers +#include +#include "XrdSsi/XrdSsiLogger.hh" + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/Method.h" +#include "memman/MemMan.h" +#include "memman/MemManNone.h" +#include "mysql/MySqlConfig.h" +#include "mysql/MySqlConnection.h" +#include "qhttp/Server.h" +#include "sql/SqlConnection.h" +#include "sql/SqlConnectionFactory.h" +#include "util/common.h" +#include "util/FileMonitor.h" +#include "util/HoldTrack.h" +#include "wbase/Base.h" +#include "wbase/FileChannelShared.h" +#include "wconfig/WorkerConfig.h" +#include "wconfig/WorkerConfigError.h" +#include "wcontrol/Foreman.h" +#include "wcontrol/SqlConnMgr.h" +#include "wpublish/ChunkInventory.h" +#include "wsched/BlendScheduler.h" +#include "wsched/FifoScheduler.h" +#include "wsched/GroupScheduler.h" +#include "wsched/ScanScheduler.h" +#include "xrdsvc/HttpSvc.h" +#include "xrdsvc/XrdName.h" + +using namespace lsst::qserv; +using namespace nlohmann; +using namespace std; +using namespace std::literals; + +class XrdPosixCallBack; // Forward. + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiService"); + +// add LWP to MDC in log messages +void initMDC() { LOG_MDC("LWP", to_string(lsst::log::lwpID())); } +int dummyInitMDC = LOG_MDC_INIT(initMDC); + +std::shared_ptr makeChunkInventory(mysql::MySqlConfig const& mySqlConfig) { + xrdsvc::XrdName x; + if (!mySqlConfig.dbName.empty()) { + LOGS(_log, LOG_LVL_FATAL, "dbName must be empty to prevent accidental context"); + throw runtime_error("dbName must be empty to prevent accidental context"); + } + auto conn = sql::SqlConnectionFactory::make(mySqlConfig); + assert(conn); + auto inventory = make_shared(x.getName(), conn); + ostringstream os; + os << "Paths exported: "; + inventory->dbgPrint(os); + LOGS(_log, LOG_LVL_DEBUG, os.str()); + return inventory; +} + +/** + * This function will keep periodically updating worker's info in the Replication + * System's Registry. + * @param id The unique identifier of a worker to be registered. + * @note The thread will terminate the process if the registraton request to the Registry + * was explicitly denied by the service. This means the application may be misconfigured. + * Transient communication errors when attempting to connect or send requests to + * the Registry will be posted into the log stream and ignored. + */ +void registryUpdateLoop(string const& id) { + auto const workerConfig = wconfig::WorkerConfig::instance(); + auto const method = http::Method::POST; + string const url = "http://" + workerConfig->replicationRegistryHost() + ":" + + to_string(workerConfig->replicationRegistryPort()) + "/qserv-worker"; + vector const headers = {"Content-Type: application/json"}; + json const request = json::object({{"version", http::MetaModule::version}, + {"instance_id", workerConfig->replicationInstanceId()}, + {"auth_key", workerConfig->replicationAuthKey()}, + {"worker", + {{"name", id}, + {"management-port", workerConfig->replicationHttpPort()}, + {"management-host-name", util::get_current_host_fqdn()}}}}); + string const requestContext = + "SsiService: '" + http::method2string(method) + "' request to '" + url + "'"; + http::Client client(method, url, request.dump(), headers); + while (true) { + try { + json const response = client.readAsJson(); + if (0 == response.at("success").get()) { + string const error = response.at("error").get(); + LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); + abort(); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + } + this_thread::sleep_for(chrono::seconds(max(1U, workerConfig->replicationRegistryHearbeatIvalSec()))); + } +} + +} // namespace + +namespace lsst::qserv::xrdsvc { + +SsiService::SsiService(XrdSsiLogger* log) { + LOGS(_log, LOG_LVL_DEBUG, "SsiService starting..."); + + util::HoldTrack::setup(10min); + + auto const mySqlConfig = wconfig::WorkerConfig::instance()->getMySqlConfig(); + if (not mysql::MySqlConnection::checkConnection(mySqlConfig)) { + LOGS(_log, LOG_LVL_FATAL, "Unable to connect to MySQL using configuration:" << mySqlConfig); + throw wconfig::WorkerConfigError("Unable to connect to MySQL"); + } + auto const workerConfig = wconfig::WorkerConfig::instance(); + string cfgMemMan = workerConfig->getMemManClass(); + memman::MemMan::Ptr memMan; + if (cfgMemMan == "MemManReal") { + // Default to 1 gigabyte + uint64_t memManSize = workerConfig->getMemManSizeMb() * 1000000; + LOGS(_log, LOG_LVL_DEBUG, + "Using MemManReal with memManSizeMb=" << workerConfig->getMemManSizeMb() + << " location=" << workerConfig->getMemManLocation()); + memMan = shared_ptr( + memman::MemMan::create(memManSize, workerConfig->getMemManLocation())); + } else if (cfgMemMan == "MemManNone") { + memMan = make_shared(1, false); + } else if (cfgMemMan == "MemManNoneRelaxed") { + bool const alwaysLock = true; + memMan = make_shared(1, alwaysLock); + } else { + LOGS(_log, LOG_LVL_ERROR, "Unrecognized memory manager " << cfgMemMan); + throw wconfig::WorkerConfigError("Unrecognized memory manager."); + } + + // Set thread pool size. + unsigned int poolSize = ranges::max({wsched::BlendScheduler::getMinPoolSize(), + workerConfig->getThreadPoolSize(), thread::hardware_concurrency()}); + + unsigned int maxPoolThreads = max(workerConfig->getMaxPoolThreads(), poolSize); + + // poolSize should be greater than either GroupScheduler::maxThreads or ScanScheduler::maxThreads + unsigned int maxThread = poolSize; + int maxReserve = 2; + auto group = make_shared("SchedGroup", maxThread, maxReserve, + workerConfig->getMaxGroupSize(), + wsched::SchedulerBase::getMaxPriority()); + + int const fastest = lsst::qserv::protojson::ScanInfo::Rating::FASTEST; + int const fast = lsst::qserv::protojson::ScanInfo::Rating::FAST; + int const medium = lsst::qserv::protojson::ScanInfo::Rating::MEDIUM; + int const slow = lsst::qserv::protojson::ScanInfo::Rating::SLOW; + int const slowest = lsst::qserv::protojson::ScanInfo::Rating::SLOWEST; + double fastScanMaxMinutes = (double)workerConfig->getScanMaxMinutesFast(); + double medScanMaxMinutes = (double)workerConfig->getScanMaxMinutesMed(); + double slowScanMaxMinutes = (double)workerConfig->getScanMaxMinutesSlow(); + double snailScanMaxMinutes = (double)workerConfig->getScanMaxMinutesSnail(); + int maxTasksBootedPerUserQuery = workerConfig->getMaxTasksBootedPerUserQuery(); + int maxConcurrentBootedTasks = workerConfig->getMaxConcurrentBootedTasks(); + vector scanSchedulers{ + make_shared("SchedSlow", maxThread, workerConfig->getMaxReserveSlow(), + workerConfig->getPrioritySlow(), + workerConfig->getMaxActiveChunksSlow(), memMan, medium + 1, + slow, slowScanMaxMinutes), + make_shared("SchedFast", maxThread, workerConfig->getMaxReserveFast(), + workerConfig->getPriorityFast(), + workerConfig->getMaxActiveChunksFast(), memMan, fastest, fast, + fastScanMaxMinutes), + make_shared( + "SchedMed", maxThread, workerConfig->getMaxReserveMed(), workerConfig->getPriorityMed(), + workerConfig->getMaxActiveChunksMed(), memMan, fast + 1, medium, medScanMaxMinutes), + }; + + auto snail = make_shared( + "SchedSnail", maxThread, workerConfig->getMaxReserveSnail(), workerConfig->getPrioritySnail(), + workerConfig->getMaxActiveChunksSnail(), memMan, slow + 1, slowest, snailScanMaxMinutes); + + wpublish::QueriesAndChunks::Ptr queries = wpublish::QueriesAndChunks::setupGlobal( + chrono::minutes(5), chrono::minutes(2), maxTasksBootedPerUserQuery, maxConcurrentBootedTasks, + false); + wsched::BlendScheduler::Ptr blendSched = make_shared( + "BlendSched", queries, maxThread, group, snail, scanSchedulers); + blendSched->setPrioritizeByInFlight(workerConfig->getPrioritizeByInFlight()); + queries->setBlendScheduler(blendSched); + + unsigned int requiredTasksCompleted = workerConfig->getRequiredTasksCompleted(); + queries->setRequiredTasksCompleted(requiredTasksCompleted); + + int const maxSqlConn = workerConfig->getMaxSqlConnections(); + int const resvInteractiveSqlConn = workerConfig->getReservedInteractiveSqlConnections(); + auto sqlConnMgr = make_shared(maxSqlConn, maxSqlConn - resvInteractiveSqlConn); + LOGS(_log, LOG_LVL_WARN, "config sqlConnMgr" << *sqlConnMgr); + LOGS(_log, LOG_LVL_WARN, "maxPoolThreads=" << maxPoolThreads); + + int qPoolSize = workerConfig->getQPoolSize(); + int maxPriority = workerConfig->getQPoolMaxPriority(); + string vectRunSizesStr = workerConfig->getQPoolRunSizes(); + string vectMinRunningSizesStr = workerConfig->getQPoolMinRunningSizes(); + + _foreman = wcontrol::Foreman::create(blendSched, poolSize, maxPoolThreads, mySqlConfig, queries, + ::makeChunkInventory(mySqlConfig), sqlConnMgr, qPoolSize, + maxPriority, vectRunSizesStr, vectMinRunningSizesStr); + + // Watch to see if the log configuration is changed. + // If LSST_LOG_CONFIG is not defined, there's no good way to know what log + // configuration file is in use. + string logConfigFile = std::getenv("LSST_LOG_CONFIG"); + if (logConfigFile == "") { + LOGS(_log, LOG_LVL_ERROR, + "FileMonitor LSST_LOG_CONFIG was blank, no log configuration file to watch."); + } else { + LOGS(_log, LOG_LVL_ERROR, "logConfigFile=" << logConfigFile); + _logFileMonitor = make_shared(logConfigFile); + } + + // Garbage collect unclaimed result files (if any). + // ATTENTION: this is the blocking operation since it needs to be run before accepting + // new queries to ensure that worker had sufficient resources to process those. + if (workerConfig->resultsCleanUpOnStart()) { + wbase::FileChannelShared::cleanUpResultsOnWorkerRestart(); + } + + // Start the control server for processing worker management requests sent + // by the Replication System. Update the port number in the configuration + // in case if the server is run on the dynamically allocated port. + _controlHttpSvc = HttpSvc::create(_foreman, workerConfig->replicationHttpPort(), + workerConfig->getCzarComNumHttpThreads()); + + auto const port = _controlHttpSvc->start(); + workerConfig->setReplicationHttpPort(port); + + // Begin periodically updating worker's status in the Replication System's registry + // in the detached thread. This will continue before the application gets terminated. + thread registryUpdateThread(::registryUpdateLoop, _foreman->chunkInventory()->id()); + registryUpdateThread.detach(); +} + +SsiService::~SsiService() { + LOGS(_log, LOG_LVL_DEBUG, "SsiService dying."); + _controlHttpSvc->stop(); +} + +void SsiService::ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) { + LOGS(_log, LOG_LVL_ERROR, "SsiService::ProcessRequest got called"); +} + +} // namespace lsst::qserv::xrdsvc From 43e98bcc3199863da65c2b3c5166111162701a97 Mon Sep 17 00:00:00 2001 From: Igor Gaponenko Date: Fri, 5 Apr 2024 00:18:15 +0000 Subject: [PATCH 04/15] Extended transient API of QMeta to read workers-to-chunks map from database --- python/lsst/qserv/admin/itest.py | 1 - src/ccontrol/UserQueryFactory.cc | 3 + src/ccontrol/UserQuerySelect.cc | 4 +- src/ccontrol/UserQuerySelect.h | 1 + src/czar/Czar.cc | 12 +- src/czar/CzarChunkMap.cc | 2 + src/czar/CzarChunkMap.h | 17 ++ src/czar/CzarRegistry.cc | 11 + src/czar/CzarRegistry.h | 9 + src/czar/testCzar.cc | 4 + src/qdisp/CMakeLists.txt | 1 + src/qdisp/Executive.cc | 67 +++++ src/qdisp/Executive.h | 9 + src/qdisp/JobDescription.h | 2 + src/qdisp/JobQuery.cc | 20 ++ src/qdisp/JobQuery.h | 3 +- src/qdisp/QueryRequest.cc | 477 ------------------------------- src/qdisp/QueryRequest.h | 180 ------------ src/qdisp/UberJob.cc | 1 + src/qmeta/CMakeLists.txt | 8 + src/qmeta/QMeta.h | 60 ++++ src/qmeta/QMetaMysql.cc | 12 + src/qmeta/QMetaMysql.h | 4 + src/xrdsvc/SsiRequest.cc | 410 ++++++++++++++++++++++++++ 24 files changed, 656 insertions(+), 662 deletions(-) delete mode 100644 src/qdisp/QueryRequest.cc delete mode 100644 src/qdisp/QueryRequest.h create mode 100644 src/xrdsvc/SsiRequest.cc diff --git a/python/lsst/qserv/admin/itest.py b/python/lsst/qserv/admin/itest.py index 900783087b..cb7d9c5b04 100644 --- a/python/lsst/qserv/admin/itest.py +++ b/python/lsst/qserv/admin/itest.py @@ -1084,7 +1084,6 @@ def compare_query_results(run_cases: list[str], outputs_dir: str) -> list[ITestC if not os.path.exists(os.path.join(outputs_dir, case)): _log.warn("There are no query results to compare for %s", case) continue - comparisons = ( (query_mode_mysql, query_mode_qserv_attached), (query_mode_mysql, query_mode_qserv_detached), diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 71843811e4..2ced51d111 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -267,6 +267,9 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st // First check for SUBMIT and strip it std::string query = aQuery; + // TODO: DM-43386 need to have WorkerChunkMap info at this point + // &&& + std::string stripped; bool async = false; if (UserQueryType::isSubmit(query, stripped)) { diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 82a01af775..fa04c8bc6f 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -86,8 +86,7 @@ #include "global/LogContext.h" #include "proto/worker.pb.h" #include "qdisp/Executive.h" -#include "qdisp/JobQuery.h" -#include "qmeta/MessageStore.h" +#include "qdisp/MessageStore.h" #include "qmeta/QMeta.h" #include "qmeta/Exceptions.h" #include "qmeta/QMeta.h" @@ -235,6 +234,7 @@ void UserQuerySelect::submit() { LOGS(_log, LOG_LVL_ERROR, "UserQuerySelect::submit() executive is null at start"); return; } + _qSession->finalize(); // Using the QuerySession, generate query specs (text, db, chunkId) and then diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index 6447f8fcd5..11c50e70fc 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -110,6 +110,7 @@ class UserQuerySelect : public UserQuery { /// Begin execution of the query over all ChunkSpecs added so far. void submit() override; + void submitOld(); //&&& /// Wait until the query has completed execution. /// @return the final execution state. diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index c56d976f9e..cb118f676e 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -71,7 +71,6 @@ using namespace std; namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.Czar"); } // anonymous namespace @@ -174,6 +173,12 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->czarId); + try { + _czarChunkMap = CzarChunkMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + } catch (ChunkMapException const& exc) { + LOGS(_log, LOG_LVL_WARN, string(__func__) + " failed to create CzarChunkMap " + exc.what()); + } + // Tell workers to cancel any queries that were submitted before this restart of Czar. // Figure out which query (if any) was recorded in Czar databases before the restart. // The id will be used as the high-watermark for queries that need to be cancelled. @@ -255,6 +260,11 @@ Czar::~Czar() { LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar() end"); } +Czar::~Czar() { + LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar()"); + cout << "&&& Czar::~Czar()" << endl; +} + SubmitResult Czar::submitQuery(string const& query, map const& hints) { LOGS(_log, LOG_LVL_DEBUG, "New query: " << query << ", hints: " << util::printable(hints)); diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index c1ebad63eb..f16f82092b 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -25,6 +25,7 @@ // System headers #include +#include // &&& del // LSST headers #include "lsst/log/Log.h" @@ -288,6 +289,7 @@ string CzarChunkMap::WorkerChunksData::dump() const { return os.str(); } + CzarFamilyMap::Ptr CzarFamilyMap::create(std::shared_ptr const& qmeta) { // There's nothing the czar can do until with user queries until there's been at least // one successful read of the database family tables, as the czar doesn't know where to find anything. diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index 8e00ac001d..800b5671a8 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -34,6 +34,7 @@ // Qserv headers #include "global/clock_defs.h" +// #include "qmeta/QMeta.h" &&& #include "util/Issue.h" namespace lsst::qserv::qmeta { @@ -51,6 +52,20 @@ class ChunkMapException : public util::Issue { ChunkMapException(Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} }; + +/// This class is used to organize worker chunk table information so that it +/// can be used to send jobs to the appropriate worker and inform workers +/// what chunks they can expect to handle in shared scans. +/// The data for the maps is provided by the Replicator and stored in the +/// QMeta database. +/// When the data is changed, there is a timestamp that is updated, which +/// will cause new maps to be made by this class. +/// +/// The maps generated are constant objects stored with shared pointers. As +/// such, it should be possible for numerous threads to use each map +/// simultaneously provided they have their own pointers to the maps. +/// The pointers to the maps are mutex protected to safely allow map updates. +/// /// The czar is expected to heavily use the /// `getMaps() -> WorkerChunkMap -> getSharedScanChunkMap()` /// to send jobs to workers, as that gets an ordered list of all chunks @@ -81,6 +96,8 @@ class CzarChunkMap { ~CzarChunkMap(); + ~CzarChunkMap(); + class WorkerChunksData; /// Essentially a structure for storing data about which tables and workers are associated with this diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index e81b0e168c..34826a0ecd 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -61,13 +61,18 @@ CzarRegistry::CzarRegistry(cconfig::CzarConfig::Ptr const& czarConfig, } CzarRegistry::~CzarRegistry() { + cout << "&&& CzarRegistry::~CzarRegistry a" << endl; _loop = false; if (_czarHeartbeatThrd.joinable()) { + cout << "&&& CzarRegistry::~CzarRegistry a1" << endl; _czarHeartbeatThrd.join(); } + cout << "&&& CzarRegistry::~CzarRegistry b" << endl; if (_czarWorkerInfoThrd.joinable()) { + cout << "&&& CzarRegistry::~CzarRegistry b1" << endl; _czarWorkerInfoThrd.join(); } + cout << "&&& CzarRegistry::~CzarRegistry end" << endl; } protojson::WorkerContactInfo::WCMapPtr CzarRegistry::getWorkerContactMap() const { @@ -75,7 +80,9 @@ protojson::WorkerContactInfo::WCMapPtr CzarRegistry::getWorkerContactMap() const return _contactMap; } + void CzarRegistry::_registryUpdateLoop() { + cout << "&&& CzarRegistry::_registryUpdateLoop a" << endl; auto const method = http::Method::POST; string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + to_string(_czarConfig->replicationRegistryPort()) + "/czar"; @@ -106,9 +113,11 @@ void CzarRegistry::_registryUpdateLoop() { } this_thread::sleep_for(chrono::seconds(max(1U, _czarConfig->replicationRegistryHearbeatIvalSec()))); } + cout << "&&& CzarRegistry::_registryUpdateLoop end" << endl; } void CzarRegistry::_registryWorkerInfoLoop() { + cout << "&&& CzarRegistry::_registryWorkerInfoLoop a" << endl; // Get worker information from the registry string const replicationInstanceId = _czarConfig->replicationInstanceId(); string const replicationAuthKey = _czarConfig->replicationAuthKey(); @@ -151,6 +160,7 @@ void CzarRegistry::_registryWorkerInfoLoop() { } this_thread::sleep_for(chrono::seconds(15)); } + cout << "&&& CzarRegistry::_registryWorkerInfoLoop end" << endl; } protojson::WorkerContactInfo::WCMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { @@ -180,6 +190,7 @@ protojson::WorkerContactInfo::WCMapPtr CzarRegistry::_buildMapFromJson(nlohmann: return wMap; } + bool CzarRegistry::_compareMapContactInfo(protojson::WorkerContactInfo::WCMap const& other) const { VMUTEX_HELD(_cmapMtx); if (_contactMap == nullptr) { diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index 08d24a7bcc..dd73e08133 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -38,6 +38,7 @@ #include "global/clock_defs.h" #include "util/Mutex.h" + namespace lsst::qserv::cconfig { class CzarConfig; } // namespace lsst::qserv::cconfig @@ -88,11 +89,19 @@ class CzarRegistry { /// `deleteWorkerResults` is true. void endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults); + /// Return _contactMap, the object that the returned pointer points to is + /// constant and no attempts should be made to change it. + WorkerContactMapPtr getWorkerContactMap() { + std::lock_guard lockG(_mapMtx); + return _contactMap; + } + private: CzarRegistry() = delete; CzarRegistry(std::shared_ptr const& czarConfig, std::shared_ptr const& activeWorkerMap); + /// This function will keep periodically updating Czar's info in the Replication System's Registry /// until _loop is set to false. /// Communications problems are logged but ignored. This should probably change. diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index 45f42f8318..3a3c761e36 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -89,6 +89,7 @@ qmeta::QMetaChunkMap convertJsonToChunkMap(nlohmann::json const& jsChunks) { BOOST_AUTO_TEST_CASE(CzarChunkMap) { // Each chunk only occurs on one worker + cerr << "&&& a" << endl; string test1 = R"( { "ce1c1b79-e6fb-11ee-a46b-0242c0a80308": @@ -125,6 +126,7 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } } )"; + cerr << "&&& b " << test1 << endl; /// 3 workers, each containing all chunks. string test2 = R"( @@ -185,6 +187,7 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } } )"; + cerr << "&&& c" << endl; auto dbToFamily = make_shared(); czar::CzarFamilyMap czFamMap(dbToFamily); @@ -195,6 +198,7 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { czar::CzarFamilyMap::verify(familyMap); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 1 passed"); + cerr << "&&& g" << endl; auto jsTest2 = nlohmann::json::parse(test2); qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2, true); diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index f15024d7a2..498a0fcf0c 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -5,6 +5,7 @@ target_sources(qdisp PRIVATE ChunkMeta.cc CzarStats.cc Executive.cc + JobBase.cc JobDescription.cc JobQuery.cc UberJob.cc diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index b2b6be3469..6cc8044662 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -210,6 +210,7 @@ UberJob::Ptr Executive::findUberJob(UberJobId ujId) { /// Add a new job to executive queue, if not already in. Not thread-safe. /// JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::add start"); JobQuery::Ptr jobQuery; { // Create the JobQuery and put it in the map. @@ -278,10 +279,21 @@ void Executive::addAndQueueUberJob(shared_ptr const& uj) { } else { _qdispPool->queCmd(cmd, 1); } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueJobStart end"); +} + +void Executive::runUberJob(std::shared_ptr const& uberJob) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); + bool started = uberJob->runUberJob(); + if (!started && isLimitRowComplete()) { + uberJob->callMarkCompleteFunc(false); + } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob end"); } void Executive::waitForAllJobsToStart() { LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart"); + LOGS(_log, LOG_LVL_WARN, "&&& waitForAllJobsToStart start"); // Wait for each command to start. while (true) { bool empty = _jobStartCmdList.empty(); @@ -291,6 +303,7 @@ void Executive::waitForAllJobsToStart() { cmd->waitComplete(); } LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart done"); + LOGS(_log, LOG_LVL_WARN, "&&& waitForAllJobsToStart end"); } Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { @@ -341,6 +354,60 @@ void Executive::addMultiError(int errorCode, std::string const& errorMsg, int er } } +Executive::ChunkIdJobMapType& Executive::getChunkJobMapAndInvalidate() { // &&& + lock_guard lck(_chunkToJobMapMtx); + if (_chunkToJobMapInvalid.exchange(true)) { + throw util::Bug(ERR_LOC, "getChunkJobMapInvalidate called when map already invalid"); + } + return _chunkToJobMap; +} + +void Executive::addUberJobs(std::vector> const& uJobsToAdd) { // &&& + lock_guard lck(_uberJobsMtx); + for (auto const& uJob : uJobsToAdd) { + _uberJobs.push_back(uJob); + } +} + +bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& + + lock_guard lock(_cancelled.getMutex()); + + // If this has been cancelled, then return false. + // + if (_cancelled) return false; + + // Construct a temporary resource object to pass to ProcessRequest(). + // Affinity should be meaningless here as there should only be one instance of each worker. + XrdSsiResource::Affinity affinity = XrdSsiResource::Affinity::Default; + LOGS(_log, LOG_LVL_INFO, "&&& startUberJob uJob->workerResource=" << uJob->getWorkerResource()); + XrdSsiResource uJobResource(uJob->getWorkerResource(), "", uJob->getIdStr(), "", 0, affinity); + + // Now construct the actual query request and tie it to the jobQuery. The + // shared pointer is used by QueryRequest to keep itself alive, sloppy design. + // Note that JobQuery calls StartQuery that then calls JobQuery, yech! + // + QueryRequest::Ptr qr = QueryRequest::create(uJob); + uJob->setQueryRequest(qr); + + // Start the query. The rest is magically done in the background. + // + getXrdSsiService()->ProcessRequest(*(qr.get()), uJobResource); + return true; +} + +JobQuery::Ptr Executive::getSharedPtrForRawJobPtr(JobQuery* jqRaw) { //&&& + assert(jqRaw != nullptr); + int jobId = jqRaw->getIdInt(); + lock_guard lockJobMap(_jobMapMtx); + auto iter = _jobMap.find(jobId); + if (iter == _jobMap.end()) { + throw util::Bug(ERR_LOC, "Could not find the entry for jobId=" + to_string(jobId)); + } + JobQuery::Ptr jq = iter->second; + return jq; +} + /// Add a JobQuery to this Executive. /// Return true if it was successfully added to the map. /// diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index bce5c670ec..63506f35f8 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -329,6 +329,15 @@ class Executive : public std::enable_shared_from_this { std::map> _uberJobsMap; mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. + // Add a job to the _chunkToJobMap //&&&uj + void _addToChunkJobMap(std::shared_ptr const& job); //&&&uj + /// _chunkToJobMap is created once and then destroyed when used. + std::atomic _chunkToJobMapInvalid{false}; ///< true indicates the map is no longer valid. //&&&uj + std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap //&&&uj + ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId //&&&uj + std::vector> _uberJobs; ///< List of UberJobs //&&&uj + std::mutex _uberJobsMtx; ///< protects _uberJobs. //&&&uj + /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. std::atomic _rowLimitComplete{false}; diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 7c28a1698e..acee88cdfb 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -90,6 +90,8 @@ class JobDescription { void resetJsForWorker() { _jsForWorker.reset(); } + bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj + friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); private: diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 21fcbd14e7..3fd1ce1bbf 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -43,6 +43,7 @@ using namespace std; namespace lsst::qserv::qdisp { +/* &&& JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, qmeta::JobStatus::Ptr const& jobStatus, QueryId qid) : _executive(executive), @@ -52,6 +53,21 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } +*/ + +JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, + JobStatus::Ptr const& jobStatus, shared_ptr const& markCompleteFunc, + QueryId qid) + : JobBase(), + _executive(executive), + _jobDescription(jobDescription), + _markCompleteFunc(markCompleteFunc), + _jobStatus(jobStatus), + _qid(qid), + _idStr(QueryIdHelper::makeIdStr(qid, getIdInt())) { + _qdispPool = executive->getQdispPool(); + LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); +} JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_TRACE, "~JobQuery QID=" << _idStr); } @@ -110,6 +126,10 @@ bool JobQuery::_setUberJobId(UberJobId ujId) { return true; } +ostream& JobQuery::dumpOS(ostream& os) const { + return os << "{" << getIdStr() << _jobDescription << " " << _jobStatus << "}"; +} + bool JobQuery::unassignFromUberJob(UberJobId ujId) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); VMUTEX_NOT_HELD(_jqMtx); diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 10e2bd59af..4f37dcac89 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -33,6 +33,7 @@ // Qserv headers #include "qdisp/Executive.h" +#include "qdisp/JobBase.h" #include "qdisp/JobDescription.h" #include "qdisp/ResponseHandler.h" #include "util/InstanceCount.h" @@ -64,7 +65,7 @@ class JobQuery { qmeta::JobStatus::Ptr getStatus() { return _jobStatus; } bool cancel(bool superfluous = false); - bool isQueryCancelled(); + bool isQueryCancelled() override; std::shared_ptr getExecutive() { return _executive.lock(); } diff --git a/src/qdisp/QueryRequest.cc b/src/qdisp/QueryRequest.cc deleted file mode 100644 index 93304a3a8a..0000000000 --- a/src/qdisp/QueryRequest.cc +++ /dev/null @@ -1,477 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2016 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -/** - * @file - * - * @brief QueryRequest. XrdSsiRequest impl for czar query dispatch - * - * @author Daniel L. Wang, SLAC - */ - -// Class header -#include "qdisp/QdispPool.h" -#include "qdisp/QueryRequest.h" - -// System headers -#include -#include -#include - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "cconfig/CzarConfig.h" -#include "czar/Czar.h" -#include "qdisp/CzarStats.h" -#include "global/LogContext.h" -#include "http/Client.h" -#include "http/ClientConnPool.h" -#include "http/Method.h" -#include "proto/worker.pb.h" -#include "qdisp/JobStatus.h" -#include "qdisp/ResponseHandler.h" -#include "util/Bug.h" -#include "util/common.h" -#include "util/InstanceCount.h" -#include "util/Timer.h" - -namespace http = lsst::qserv::http; -namespace qdisp = lsst::qserv::qdisp; - -using namespace std; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.QueryRequest"); - -/** - * The RAII class for removing the HTTP file (if it still exist). - * @note Errors are logged, but not reported to the caller. - */ -class HttpFileRemoverRAII { -public: - HttpFileRemoverRAII() = delete; - HttpFileRemoverRAII(HttpFileRemoverRAII const&) = delete; - HttpFileRemoverRAII& operator=(HttpFileRemoverRAII const&) = delete; - HttpFileRemoverRAII(string const& httpUrl) : _httpUrl(httpUrl) {} - - ~HttpFileRemoverRAII() { - if (_httpUrl.empty()) return; - string const noClientData; - vector const noClientHeaders; - try { - http::Client remover(http::Method::DELETE, _httpUrl, noClientData, noClientHeaders, - qdisp::QueryRequest::makeHttpClientConfig(), - qdisp::QueryRequest::getHttpConnPool()); - remover.read([](char const*, size_t) {}); - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, - "HttpFileRemoverRAII failed to remove " << _httpUrl << ", ex: " << ex.what()); - } - } - -private: - string const _httpUrl; -}; -} // namespace - -namespace lsst::qserv::qdisp { - -shared_ptr QueryRequest::_httpConnPool; -mutex QueryRequest::_httpConnPoolMutex; - -http::ClientConfig QueryRequest::makeHttpClientConfig() { - http::ClientConfig clientConfig; - clientConfig.httpVersion = CURL_HTTP_VERSION_1_1; // same as in qhttp - clientConfig.bufferSize = CURL_MAX_READ_SIZE; // 10 MB in the current version of libcurl - clientConfig.tcpKeepAlive = true; - clientConfig.tcpKeepIdle = 5; // the default is 60 sec - clientConfig.tcpKeepIntvl = 5; // the default is 60 sec - return clientConfig; -} - -shared_ptr const& QueryRequest::getHttpConnPool() { - lock_guard const lock(_httpConnPoolMutex); - if (nullptr == _httpConnPool) { - _httpConnPool = make_shared( - cconfig::CzarConfig::instance()->getResultMaxHttpConnections()); - } - return _httpConnPool; -} -QueryRequest::QueryRequest(JobQuery::Ptr const& jobQuery) - : _jobQuery(jobQuery), - _qid(jobQuery->getQueryId()), - _jobid(jobQuery->getIdInt()), - _jobIdStr(jobQuery->getIdStr()), - _qdispPool(_jobQuery->getQdispPool()) { - QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); - LOGS(_log, LOG_LVL_TRACE, "New QueryRequest"); -} - -QueryRequest::~QueryRequest() { - QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); - LOGS(_log, LOG_LVL_TRACE, __func__); - if (!_finishedCalled.exchange(true)) { - LOGS(_log, LOG_LVL_WARN, __func__ << " cleaning up calling Finished"); - bool ok = Finished(); - if (!ok) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " Finished NOT ok"); - } - } -} - -// content of request data -char* QueryRequest::GetRequest(int& requestLength) { - QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); - lock_guard lock(_finishStatusMutex); - auto jq = _jobQuery; - if (_finishStatus != ACTIVE || jq == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " called after job finished (cancelled?)"); - requestLength = 0; - return const_cast(""); - } - requestLength = jq->getDescription()->payload().size(); - LOGS(_log, LOG_LVL_DEBUG, "Requesting, payload size: " << requestLength); - // Andy promises that his code won't corrupt it. - return const_cast(jq->getDescription()->payload().data()); -} - -// Must not throw exceptions: calling thread cannot trap them. -// Callback function for XrdSsiRequest. -// -bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo const& rInfo) { - QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); - LOGS(_log, LOG_LVL_DEBUG, "workerName=" << GetEndPoint() << " " << __func__); - string errorDesc = _jobIdStr + " "; - if (isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, __func__ << " job already cancelled"); - cancel(); // calls _errorFinish() - return true; - } - - // Make a copy of the _jobQuery shared_ptr in case _jobQuery gets reset by a call to cancel() - auto jq = _jobQuery; - { - lock_guard lock(_finishStatusMutex); - if ((_finishStatus != ACTIVE) || (jq == nullptr)) { - LOGS(_log, LOG_LVL_WARN, __func__ << " called after job finished (cancelled?)"); - return true; - } - } - if (eInfo.hasError()) { - ostringstream os; - os << _jobIdStr << __func__ << " request failed " << getSsiErr(eInfo, nullptr) << " " - << GetEndPoint(); - jq->getDescription()->respHandler()->errorFlush(os.str(), -1); - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_ERROR, "SSI"); - _errorFinish(); - return true; - } - - string responseTypeName; // for error reporting - switch (rInfo.rType) { - case XrdSsiRespInfo::isNone: - responseTypeName = "isNone"; - break; - case XrdSsiRespInfo::isData: - if (string(rInfo.buff, rInfo.blen) == "MockResponse") { - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::COMPLETE, "MOCK"); - _finish(); - return true; - } else if (rInfo.blen == 0) { - // Metadata-only responses for the file-based protocol should not have any data - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); - return _importResultFile(jq); - } - responseTypeName = "isData"; - break; - case XrdSsiRespInfo::isError: - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_ERROR, "SSI", rInfo.eNum, - string(rInfo.eMsg)); - return _importError(string(rInfo.eMsg), rInfo.eNum); - case XrdSsiRespInfo::isFile: - responseTypeName = "isFile"; - break; - case XrdSsiRespInfo::isStream: - responseTypeName = "isStream"; - break; - default: - responseTypeName = ""; - } - return _importError("Unexpected XrdSsiRespInfo.rType == " + responseTypeName, -1); -} - -/// Retrieve and process a result file using the file-based protocol -/// Uses a copy of JobQuery::Ptr instead of _jobQuery as a call to cancel() would reset _jobQuery. -bool QueryRequest::_importResultFile(JobQuery::Ptr const& jq) { - // The message needs to be parsed to extract the response summary. - int messageSize = 0; - const char* message = GetMetadata(messageSize); - - LOGS(_log, LOG_LVL_DEBUG, __func__ << " _jobIdStr=" << _jobIdStr << ", messageSize=" << messageSize); - - proto::ResponseSummary resp; - if (!(resp.ParseFromArray(message, messageSize) && resp.IsInitialized())) { - string const err = "failed to parse the response summary, messageSize=" + to_string(messageSize); - LOGS(_log, LOG_LVL_ERROR, __func__ << " " << err); - throw util::Bug(ERR_LOC, err); - } - - // The file gets removed regardless of the outcome of the merge operation. - HttpFileRemoverRAII const fileRemover(resp.fileresource_http()); - - // It's possible jq and _jobQuery differ, so need to use jq. - if (jq->isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, "QueryRequest::_processData job was cancelled."); - _errorFinish(true); - return false; - } - auto executive = jq->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { - if (executive == nullptr || executive->getCancelled()) { - LOGS(_log, LOG_LVL_WARN, "QueryRequest::_processData job was cancelled."); - } else { - int dataIgnored = (executive->incrDataIgnoredCount()); - if ((dataIgnored - 1) % 1000 == 0) { - LOGS(_log, LOG_LVL_INFO, - "QueryRequest::_processData ignoring, enough rows already " << "dataIgnored=" - << dataIgnored); - } - } - _errorFinish(true); - return false; - } - if (!jq->getDescription()->respHandler()->flush(resp)) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " not flushOk"); - _flushError(jq); - return false; - } - _totalRows += resp.rowcount(); - - // If the query meets the limit row complete complete criteria, it will start - // squashing superfluous results so the answer can be returned quickly. - // This needs to be done before marking the current job as complete. - executive->addResultRows(_totalRows); - executive->checkLimitRowComplete(); - - // At this point all data for this job have been read, there's no point in - // having XrdSsi wait for anything. - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::COMPLETE, "COMPLETE"); - _finish(); - - return true; -} - -/// Process an incoming error. -bool QueryRequest::_importError(string const& msg, int code) { - auto jq = _jobQuery; - { - lock_guard lock(_finishStatusMutex); - if (_finishStatus != ACTIVE || jq == nullptr) { - LOGS(_log, LOG_LVL_WARN, - "QueryRequest::_importError code=" << code << " msg=" << msg << " not passed"); - return false; - } - jq->getDescription()->respHandler()->errorFlush(msg, code); - } - _errorFinish(); - return true; -} - -void QueryRequest::ProcessResponseData(XrdSsiErrInfo const& eInfo, char* buff, int blen, bool last) { - string const err = "the method has no use in this implementation of Qserv"; - LOGS(_log, LOG_LVL_ERROR, __func__ << " " << err); - throw util::Bug(ERR_LOC, err); -} - -void QueryRequest::_flushError(JobQuery::Ptr const& jq) { - ResponseHandler::Error err = jq->getDescription()->respHandler()->getError(); - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::MERGE_ERROR, "MERGE", err.getCode(), err.getMsg(), - MSG_ERROR); - _errorFinish(true); -} - -/// @return true if QueryRequest cancelled successfully. -bool QueryRequest::cancel() { - LOGS(_log, LOG_LVL_DEBUG, "QueryRequest::cancel"); - { - lock_guard lock(_finishStatusMutex); - if (_cancelled) { - LOGS(_log, LOG_LVL_DEBUG, "QueryRequest::cancel already cancelled, ignoring"); - return false; // Don't do anything if already cancelled. - } - _cancelled = true; - _retried = true; // Prevent retries. - // Only call the following if the job is NOT already done. - if (_finishStatus == ACTIVE) { - auto jq = _jobQuery; - if (jq != nullptr) jq->getStatus()->updateInfo(_jobIdStr, JobStatus::CANCEL, "CANCEL"); - } - } - return _errorFinish(true); // return true if errorFinish cancelled -} - -/// @return true if this object's JobQuery, or its Executive has been cancelled. -/// It takes time for the Executive to flag all jobs as being cancelled -bool QueryRequest::isQueryCancelled() { - auto jq = _jobQuery; - if (jq == nullptr) { - // Need to check if _jobQuery is null due to cancellation. - return isQueryRequestCancelled(); - } - return jq->isQueryCancelled(); -} - -/// @return true if QueryRequest::cancel() has been called. -/// QueryRequest::isQueryCancelled() is a much better indicator of user query cancellation. -bool QueryRequest::isQueryRequestCancelled() { - lock_guard lock(_finishStatusMutex); - return _cancelled; -} - -/// Cleanup pointers so this class can be deleted. -/// This should only be called by _finish or _errorFinish. -void QueryRequest::cleanup() { - LOGS(_log, LOG_LVL_TRACE, "QueryRequest::cleanup()"); - { - lock_guard lock(_finishStatusMutex); - if (_finishStatus == ACTIVE) { - LOGS(_log, LOG_LVL_ERROR, "QueryRequest::cleanup called before _finish or _errorFinish"); - return; - } - } - - // These need to be outside the mutex lock, or you could delete - // _finishStatusMutex before it is unlocked. - // This should reset _jobquery and _keepAlive without risk of either being deleted - // before being reset. - _jobQuery = nullptr; - _keepAlive = nullptr; -} - -/// Finalize under error conditions and retry or report completion -/// THIS FUNCTION WILL RESULT IN THIS OBJECT BEING DESTROYED, UNLESS there is -/// a local shared pointer for this QueryRequest and/or its owner JobQuery. -/// See QueryRequest::cleanup() -/// @return true if this QueryRequest object had the authority to make changes. -bool QueryRequest::_errorFinish(bool stopTrying) { - LOGS(_log, LOG_LVL_DEBUG, "_errorFinish() shouldCancel=" << stopTrying); - auto jq = _jobQuery; - { - // Running _errorFinish more than once could cause errors. - lock_guard lock(_finishStatusMutex); - if (_finishStatus != ACTIVE || jq == nullptr) { - // Either _finish or _errorFinish has already been called. - LOGS_DEBUG("_errorFinish() job no longer ACTIVE, ignoring " - << " _finishStatus=" << _finishStatus << " ACTIVE=" << ACTIVE << " jq=" << jq); - return false; - } - _finishStatus = ERROR; - } - - // Make the calls outside of the mutex lock. - LOGS(_log, LOG_LVL_DEBUG, "calling Finished(stopTrying=" << stopTrying << ")"); - bool ok = Finished(); - if (_finishedCalled.exchange(true)) { - LOGS(_log, LOG_LVL_WARN, "QueryRequest::_errorFinish Finished() already called"); - } - if (!ok) { - LOGS(_log, LOG_LVL_ERROR, "QueryRequest::_errorFinish NOT ok"); - } - - if (!_retried.exchange(true) && !stopTrying) { - // There's a slight race condition here. _jobQuery::runJob() creates a - // new QueryRequest object which will replace this one in _jobQuery. - // The replacement could show up before this one's cleanup() is called, - // so this will keep this alive. - LOGS(_log, LOG_LVL_DEBUG, "QueryRequest::_errorFinish retrying"); - _keepAlive = jq->getQueryRequest(); // shared pointer to this - if (!jq->runJob()) { - // Retry failed, nothing left to try. - LOGS(_log, LOG_LVL_DEBUG, "errorFinish retry failed"); - _callMarkComplete(false); - } - } else { - _callMarkComplete(false); - } - cleanup(); // Reset smart pointers so this object can be deleted. - return true; -} - -/// Finalize under success conditions and report completion. -/// THIS FUNCTION WILL RESULT IN THIS OBJECT BEING DESTROYED, UNLESS there is -/// a local shared pointer for this QueryRequest and/or its owner JobQuery. -/// See QueryRequest::cleanup() -void QueryRequest::_finish() { - LOGS(_log, LOG_LVL_TRACE, "QueryRequest::_finish"); - { - // Running _finish more than once would cause errors. - lock_guard lock(_finishStatusMutex); - if (_finishStatus != ACTIVE) { - // Either _finish or _errorFinish has already been called. - LOGS(_log, LOG_LVL_WARN, "QueryRequest::_finish called when not ACTIVE, ignoring"); - return; - } - _finishStatus = FINISHED; - } - - bool ok = Finished(); - if (_finishedCalled.exchange(true)) { - LOGS(_log, LOG_LVL_WARN, "QueryRequest::finish Finished() already called"); - } - if (!ok) { - LOGS(_log, LOG_LVL_ERROR, "QueryRequest::finish Finished() !ok "); - } - _callMarkComplete(true); - cleanup(); -} - -/// Inform the Executive that this query completed, and -// Call MarkCompleteFunc only once, it should only be called from _finish() or _errorFinish. -void QueryRequest::_callMarkComplete(bool success) { - if (!_calledMarkComplete.exchange(true)) { - auto jq = _jobQuery; - if (jq != nullptr) jq->getMarkCompleteFunc()->operator()(success); - } -} - -ostream& operator<<(ostream& os, QueryRequest const& qr) { - os << "QueryRequest " << qr._jobIdStr; - return os; -} - -/// @return The error text and code that SSI set. -/// if eCode != nullptr, it is set to the error code set by SSI. -string QueryRequest::getSsiErr(XrdSsiErrInfo const& eInfo, int* eCode) { - int errNum; - string errText = eInfo.Get(errNum); - if (eCode != nullptr) { - *eCode = errNum; - } - ostringstream os; - os << "SSI_Error(" << errNum << ":" << errText << ")"; - return os.str(); -} - -} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/QueryRequest.h b/src/qdisp/QueryRequest.h deleted file mode 100644 index 4a6283f368..0000000000 --- a/src/qdisp/QueryRequest.h +++ /dev/null @@ -1,180 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_QDISP_QUERYREQUEST_H -#define LSST_QSERV_QDISP_QUERYREQUEST_H - -// System headers -#include -#include -#include -#include -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiRequest.hh" - -// Local headers -#include "czar/Czar.h" -#include "qdisp/JobQuery.h" -#include "qdisp/QdispPool.h" - -// Forward declarations - -namespace lsst::qserv::http { -class ClientConfig; -class ClientConnPool; -} // namespace lsst::qserv::http - -namespace lsst::qserv::qdisp { - -/// Bad response received from SSI API -class BadResponseError : public std::exception { -public: - BadResponseError(std::string const& s_) : std::exception(), s("BadResponseError:" + s_) {} - virtual ~BadResponseError() throw() {} - virtual const char* what() const throw() { return s.c_str(); } - std::string s; -}; - -/// Error in QueryRequest -class RequestError : public std::exception { -public: - RequestError(std::string const& s_) : std::exception(), s("QueryRequest error:" + s_) {} - virtual ~RequestError() throw() {} - virtual const char* what() const throw() { return s.c_str(); } - std::string s; -}; - -/// A client implementation of an XrdSsiRequest that adapts qserv's executing -/// queries to the XrdSsi API. -/// -/// Memory allocation notes: -/// In the XrdSsi API, raw pointers are passed around for XrdSsiRequest objects, -/// and care needs to be taken to avoid deleting the request objects before -/// Finished() is called. Typically, an XrdSsiRequest subclass is allocated with -/// operator new, and passed into XrdSsi. At certain points in the transaction, -/// XrdSsi will call methods in the request object or hand back the request -/// object pointer. XrdSsi ceases interest in the object once the -/// XrdSsiRequest::Finished() completes. Generally, this would mean the -/// QueryRequest should clean itself up after calling Finished(). This requires -/// special care, because there is a cancellation function in the wild that may -/// call into QueryRequest after Finished() has been called. The cancellation -/// code is -/// designed to allow the client requester (elsewhere in qserv) to request -/// cancellation without knowledge of XrdSsi, so the QueryRequest registers a -/// cancellation function with its client that maintains a pointer to the -/// QueryRequest. After Finished(), the cancellation function must be prevented -/// from accessing the QueryRequest instance. -class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this { -public: - typedef std::shared_ptr Ptr; - - static http::ClientConfig makeHttpClientConfig(); - static std::shared_ptr const& getHttpConnPool(); - - static Ptr create(std::shared_ptr const& jobQuery) { - Ptr newQueryRequest(new QueryRequest(jobQuery)); - newQueryRequest->_keepAlive = newQueryRequest; - return newQueryRequest; - } - - virtual ~QueryRequest(); - - /// Called by SSI to get the request payload - /// @return content of request data - char* GetRequest(int& requestLength) override; - - /// Called by SSI to release the allocated request payload. As we don't - /// own the buffer, so we can't release it. Therefore, we accept the - /// default implementation that does nothing. - /// void RelRequestBuffer() override; - - /// Called by SSI when a response is ready - /// precondition: rInfo.rType != isNone - bool ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo const& rInfo) override; - - /// Called by SSI when new data is available. - void ProcessResponseData(XrdSsiErrInfo const& eInfo, char* buff, int blen, bool last) override; - - bool cancel(); - bool isQueryCancelled(); - bool isQueryRequestCancelled(); - void doNotRetry() { _retried.store(true); } - std::string getSsiErr(XrdSsiErrInfo const& eInfo, int* eCode); - void cleanup(); ///< Must be called when this object is no longer needed. - - friend std::ostream& operator<<(std::ostream& os, QueryRequest const& r); - -private: - // Private constructor to safeguard enable_shared_from_this construction. - QueryRequest(std::shared_ptr const& jobQuery); - - void _callMarkComplete(bool success); - bool _importResultFile(JobQuery::Ptr const& jq); - bool _importError(std::string const& msg, int code); - bool _errorFinish(bool stopTrying = false); - void _finish(); - void _flushError(JobQuery::Ptr const& jq); - - // All instances of the HTTP client class are members of the same pool. This allows - // connection reuse and a significant reduction of the kernel memory pressure. - // Note that the pool gets instantiated at the very first call to method getHttpConnPool() - // because the instantiation depends on the availability of the Czar configuration. - static std::shared_ptr _httpConnPool; - static std::mutex _httpConnPoolMutex; - - /// Job information. Not using a weak_ptr as Executive could drop its JobQuery::Ptr before we're done with - /// it. A call to cancel() could reset _jobQuery early, so copy or protect _jobQuery with - /// _finishStatusMutex as needed. If (_finishStatus == ACTIVE) _jobQuery should be good. - std::shared_ptr _jobQuery; - - std::atomic _retried{false}; ///< Protect against multiple retries of _jobQuery from a - /// single QueryRequest. - std::atomic _calledMarkComplete{false}; ///< Protect against multiple calls to MarkCompleteFunc - /// from a single QueryRequest. - - std::mutex _finishStatusMutex; ///< used to protect _cancelled, _finishStatus, and _jobQuery. - enum FinishStatus { ACTIVE, FINISHED, ERROR } _finishStatus{ACTIVE}; // _finishStatusMutex - bool _cancelled{false}; ///< true if cancelled, protected by _finishStatusMutex. - - std::shared_ptr _keepAlive; ///< Used to keep this object alive during race condition. - QueryId _qid = 0; // for logging - int _jobid = -1; // for logging - std::string _jobIdStr{QueryIdHelper::makeIdStr(0, 0, true)}; ///< for debugging only. - - std::atomic _finishedCalled{false}; - - QdispPool::Ptr _qdispPool; - - int64_t _totalRows = 0; ///< number of rows in query added to the result table. - - std::atomic _rowsIgnored{0}; ///< Limit log messages about rows being ignored. - std::atomic _respCount{0}; ///< number of responses created -}; - -std::ostream& operator<<(std::ostream& os, QueryRequest const& r); - -} // namespace lsst::qserv::qdisp - -#endif // LSST_QSERV_QDISP_QUERYREQUEST_H diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 00ca071965..de713dac12 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -516,3 +516,4 @@ std::string UberJob::dump() const { std::ostream& operator<<(std::ostream& os, UberJob const& uj) { return uj.dumpOS(os); } } // namespace lsst::qserv::qdisp + diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index 969971f5ed..2e13f8b458 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -23,6 +23,14 @@ target_link_libraries(qmeta PUBLIC mysqlclient_r http ) +# &&& see if there's a way to remove some target_link_libraries +# &&& ccontrol +# &&& czar +# &&& parser +# &&& rproc +# &&& qana +# &&& qproc +# &&& query install(TARGETS qmeta) diff --git a/src/qmeta/QMeta.h b/src/qmeta/QMeta.h index 05d54526bb..62f87ca4a9 100644 --- a/src/qmeta/QMeta.h +++ b/src/qmeta/QMeta.h @@ -101,6 +101,50 @@ class QMeta { */ typedef std::vector > TableNames; +#if 0 //&&& + /** + * The structure ChunkMap encapsulates a disposition of chunks at Qserv workers + * along with a time when the map was updated. + * + * Here is an example on how to using the map for getting info on all chunks in + * the given context: + * @code + * std::string const worker = "worker-001"; + * std::string const database = "LSST-DR01"; + * std::string const table = "Object"; + * + * ChunkMap const& chunkMap = ...; + * for (auto const& [chunk, size] : chunkMap[worker][database][table]) { + * ... + * } + * @endcode + */ + struct QMetaChunkMap { + /// @return 'true' if the map is empty (or constructed using the default constructor) + bool empty() const { + return workers.empty() || (std::chrono::time_point() == updateTime); + } + + // NOTE: Separate types were added here for the sake of clarity to avoid + // a definition of the unreadable nested map. + + struct ChunkInfo { + unsigned int chunk = 0; ///< The chunk number + size_t size = 0; ///< The file size (in bytes) of the chunk table + }; + typedef std::vector Chunks; ///< Collection of chunks + typedef std::map Tables; ///< tables-to-chunks + typedef std::map Databases; ///< Databases-to-tables + typedef std::map Workers; ///< Workers-to-databases + + /// The chunk disposition map for all workers. + Workers workers; + + /// The last time the map was updated (since UNIX Epoch). + TIMEPOINT updateTime; + }; +#endif //&&& + /** * Create QMeta instance from configuration dictionary. * @@ -328,6 +372,22 @@ class QMeta { std::chrono::time_point const& prevUpdateTime = std::chrono::time_point()) = 0; + /** + * Fetch the chunk map which was updated after the specified time point. + * @param prevUpdateTime The cut off time for the chunk map age. Note that the default + * value of the parameter represents the start time of the UNIX Epoch. Leaving the default + * value forces an attempt to read the map from the database if the one would exist + * in there. + * @return Return the most current chunk disposition or the empty object if the persistent + * map is older than it was requested.The result could be evaluated by calling + * method empty() on the result object. + * @throws EmptyTableError if the corresponding metadata table doesn't have any record + * @throws SqlError for any other error related to MySQL + */ + virtual QMetaChunkMap getChunkMap( + std::chrono::time_point const& prevUpdateTime = + std::chrono::time_point()) = 0; + protected: // Default constructor QMeta() {} diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index e89c702c63..67a5cb6357 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -814,10 +814,14 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c string const& table = row[2]; unsigned int chunk = lsst::qserv::stoui(row[3]); size_t const size = stoull(row[4]); +#if 1 //&&& chunkMap.workers[worker][database][table].push_back(QMetaChunkMap::ChunkInfo{chunk, size}); LOGS(_log, LOG_LVL_TRACE, "QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table << " chunk=" << chunk << " sz=" << size); +#else //&&& + chunkMap.workers[worker][database][table].push_back(ChunkMap::ChunkInfo{chunk, size}); +#endif //&&& } chunkMap.updateTime = updateTime; } catch (exception const& ex) { @@ -831,9 +835,13 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock sql::SqlErrorObject errObj; sql::SqlResults results; string const tableName = "chunkMapStatus"; +#if 1 //&&& string const query = "SELECT UNIX_TIMESTAMP(`update_time`) FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; +#else //&&& + string const query = "SELECT `update_time` FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; +#endif //&&& LOGS(_log, LOG_LVL_DEBUG, "Executing query: " << query); if (!_conn->runQuery(query, results, errObj)) { LOGS(_log, LOG_LVL_ERROR, "query failed: " << query); @@ -858,7 +866,11 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock } } +#if 1 //&&& void QMetaMysql::_addQueryMessage(QueryId queryId, qmeta::QueryMessage const& qMsg, int& cancelCount, +#else //&&& +void QMetaMysql::_addQueryMessage(QueryId queryId, qdisp::QueryMessage const& qMsg, int& cancelCount, +#endif //&&& int& completeCount, int& execFailCount, map& msgCountMap) { // Don't add duplicate messages. if (qMsg.msgSource == "DUPLICATE") return; diff --git a/src/qmeta/QMetaMysql.h b/src/qmeta/QMetaMysql.h index 0afb534be2..c7c03f9919 100644 --- a/src/qmeta/QMetaMysql.h +++ b/src/qmeta/QMetaMysql.h @@ -252,6 +252,10 @@ class QMetaMysql : public QMeta { QMetaChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = std::chrono::time_point()) override; + /// @see QMeta::getChunkMap + QMetaChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = + std::chrono::time_point()) override; + protected: /// Check that all necessary tables exist void _checkDb(); diff --git a/src/xrdsvc/SsiRequest.cc b/src/xrdsvc/SsiRequest.cc new file mode 100644 index 0000000000..19db57205c --- /dev/null +++ b/src/xrdsvc/SsiRequest.cc @@ -0,0 +1,410 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2015-2016 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include +#include +#include +#include +#include +#include + +// Third-party headers +#include "XrdSsi/XrdSsiRequest.hh" + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "global/intTypes.h" +#include "global/LogContext.h" +#include "global/ResourceUnit.h" +#include "proto/FrameBuffer.h" +#include "proto/worker.pb.h" +#include "util/InstanceCount.h" +#include "util/HoldTrack.h" +#include "util/Timer.h" +#include "wbase/FileChannelShared.h" +#include "wbase/TaskState.h" +#include "wbase/Task.h" +#include "wconfig/WorkerConfig.h" +#include "wcontrol/Foreman.h" +#include "wcontrol/ResourceMonitor.h" +#include "wpublish/ChunkInventory.h" +#include "xrdsvc/ChannelStream.h" + +namespace proto = lsst::qserv::proto; +namespace wbase = lsst::qserv::wbase; + +namespace { + +LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiRequest"); + +} // namespace + +namespace lsst::qserv::xrdsvc { + +SsiRequest::Ptr SsiRequest::newSsiRequest(std::string const& rname, + std::shared_ptr const& foreman) { + auto req = SsiRequest::Ptr(new SsiRequest(rname, foreman)); + req->_selfKeepAlive = req; + return req; +} + +SsiRequest::SsiRequest(std::string const& rname, std::shared_ptr const& foreman) + : _validator(foreman->chunkInventory()->newValidator()), _foreman(foreman), _resourceName(rname) {} + +SsiRequest::~SsiRequest() { + LOGS(_log, LOG_LVL_DEBUG, "~SsiRequest()"); + UnBindRequest(); +} + +void SsiRequest::reportError(std::string const& errStr) { + LOGS(_log, LOG_LVL_WARN, errStr); + replyError(errStr, EINVAL); + ReleaseRequestBuffer(); +} + +uint64_t countLimiter = 0; // LockupDB + +// Step 4 +/// Called by XrdSsi to actually process a request. +void SsiRequest::execute(XrdSsiRequest& req) { + LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute start"); + util::Timer t; + LOGS(_log, LOG_LVL_DEBUG, "Execute request, resource=" << _resourceName); + + char* reqData = nullptr; + int reqSize; + t.start(); + reqData = req.GetRequest(reqSize); + t.stop(); + LOGS(_log, LOG_LVL_DEBUG, "GetRequest took " << t.getElapsed() << " seconds"); + + // We bind this object to the request now. This allows us to respond at any + // time (much simpler). Though the manual forgot to say that all pending + // events will be reflected on a different thread the moment we bind the + // request; the fact allows us to use a mutex to serialize the order of + // initialization and possible early cancellation. We protect this code + // with a mutex gaurd which will be released upon exit. + // + std::lock_guard lock(_finMutex); + BindRequest(req); + + ResourceUnit ru(_resourceName); + + // Make sure the requested resource belongs to this worker + if (!(*_validator)(ru)) { + reportError("WARNING: request to the unowned resource detected:" + _resourceName); + return; + } + + auto const sendChannel = std::make_shared(shared_from_this()); + + // Process the request + switch (ru.unitType()) { + case ResourceUnit::DBCHUNK: { + LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute DBCHUNK"); + // Increment the counter of the database/chunk resources in use + _foreman->resourceMonitor()->increment(_resourceName); + + // reqData has the entire request, so we can unpack it without waiting for + // more data. + LOGS(_log, LOG_LVL_DEBUG, "Decoding TaskMsg of size " << reqSize); + auto taskMsg = std::make_shared(); + if (!taskMsg->ParseFromArray(reqData, reqSize) || !taskMsg->IsInitialized()) { + reportError("Failed to decode TaskMsg on resource db=" + ru.db() + + " chunkId=" + std::to_string(ru.chunk())); + return; + } + + QSERV_LOGCONTEXT_QUERY_JOB(taskMsg->queryid(), taskMsg->jobid()); + + if (!taskMsg->has_db() || !taskMsg->has_chunkid() || (ru.db() != taskMsg->db()) || + (ru.chunk() != taskMsg->chunkid())) { + reportError("Mismatched db/chunk in TaskMsg on resource db=" + ru.db() + + " chunkId=" + std::to_string(ru.chunk())); + return; + } + + if (not(taskMsg->has_queryid() && taskMsg->has_jobid() && taskMsg->has_scaninteractive() && + taskMsg->has_attemptcount() && taskMsg->has_czarid())) { + reportError(std::string("taskMsg missing required field ") + + " queryid:" + std::to_string(taskMsg->has_queryid()) + + " jobid:" + std::to_string(taskMsg->has_jobid()) + + " scaninteractive:" + std::to_string(taskMsg->has_scaninteractive()) + + " attemptcount:" + std::to_string(taskMsg->has_attemptcount()) + + " czarid:" + std::to_string(taskMsg->has_czarid())); + return; + } + switch (wconfig::WorkerConfig::instance()->resultDeliveryProtocol()) { + case wconfig::ConfigValResultDeliveryProtocol::XROOT: + case wconfig::ConfigValResultDeliveryProtocol::HTTP: + _channelShared = wbase::FileChannelShared::create(sendChannel, taskMsg->czarid(), + _foreman->chunkInventory()->id()); + break; + default: + throw std::runtime_error("SsiRequest::" + std::string(__func__) + + " unsupported result delivery protocol"); + } + auto const tasks = wbase::Task::createTasks(taskMsg, _channelShared, _foreman->chunkResourceMgr(), + _foreman->mySqlConfig(), _foreman->sqlConnMgr(), + _foreman->queriesAndChunks(), _foreman->httpPort()); + for (auto const& task : tasks) { + _tasks.push_back(task); + } + + // Now that the request is decoded (successfully or not), release the + // xrootd request buffer. To avoid data races, this must happen before + // the task is handed off to another thread for processing, as there is a + // reference to this SsiRequest inside the reply channel for the task, + // and after the call to BindRequest. + ReleaseRequestBuffer(); + t.start(); + _foreman->processTasks(tasks); // Queues tasks to be run later. + t.stop(); + LOGS(_log, LOG_LVL_DEBUG, + "Enqueued TaskMsg for " << ru << " in " << t.getElapsed() << " seconds"); + break; + } + case ResourceUnit::QUERY: { + LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute QUERY"); + LOGS(_log, LOG_LVL_DEBUG, "Parsing request details for resource=" << _resourceName); + proto::QueryManagement request; + try { + // reqData has the entire request, so we can unpack it without waiting for + // more data. + proto::FrameBufferView view(reqData, reqSize); + view.parse(request); + ReleaseRequestBuffer(); + } catch (proto::FrameBufferError const& ex) { + reportError("Failed to decode a query completion/cancellation command, error: " + + std::string(ex.what())); + break; + } + LOGS(_log, LOG_LVL_DEBUG, + "QueryManagement: op=" << proto::QueryManagement_Operation_Name(request.op()) + << " query_id=" << request.query_id()); + + switch (wconfig::WorkerConfig::instance()->resultDeliveryProtocol()) { + case wconfig::ConfigValResultDeliveryProtocol::XROOT: + case wconfig::ConfigValResultDeliveryProtocol::HTTP: + switch (request.op()) { + case proto::QueryManagement::CANCEL_AFTER_RESTART: + // TODO: locate and cancel the coresponding tasks, remove the tasks + // from the scheduler queues. + wbase::FileChannelShared::cleanUpResultsOnCzarRestart(request.czar_id(), + request.query_id()); + break; + case proto::QueryManagement::CANCEL: + // TODO: locate and cancel the coresponding tasks, remove the tasks + // from the scheduler queues. + wbase::FileChannelShared::cleanUpResults(request.czar_id(), request.query_id()); + break; + case proto::QueryManagement::COMPLETE: + wbase::FileChannelShared::cleanUpResults(request.czar_id(), request.query_id()); + break; + default: + reportError("QueryManagement: op=" + + proto::QueryManagement_Operation_Name(request.op()) + + " is not supported by the current implementation."); + return; + } + break; + default: + throw std::runtime_error("SsiRequest::" + std::string(__func__) + + " unsupported result delivery protocol"); + } + + // Send back the empty response since no info is expected by a caller + // for this type of requests beyond the usual error notifications (if any). + this->reply((char const*)0, 0); + break; + } + default: + LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute default"); + reportError("Unexpected unit type '" + std::to_string(ru.unitType()) + + "', resource name: " + _resourceName); + break; + } + + // Note that upon exit the _finMutex will be unlocked allowing Finished() + // to actually do something once everything is actually setup. +} + +/// Called by SSI to free resources. +void SsiRequest::Finished(XrdSsiRequest& req, XrdSsiRespInfo const& rinfo, bool cancel) { // Step 8 + util::HoldTrack::Mark markA(ERR_LOC, "SsiRequest::Finished start"); + if (cancel) { + // Either the czar of xrootd has decided to cancel the Job. + // Try to cancel all of the tasks, if there are any. + for (auto&& wTask : _tasks) { + auto task = wTask.lock(); + if (task != nullptr) { + task->cancel(); + } + } + } + + // This call is sync (blocking). + // client finished retrieving response, or cancelled. + // release response resources (e.g. buf) + // But first we must make sure that request setup completed (i.e execute()) by + // locking _finMutex. + { + std::lock_guard finLock(_finMutex); + // Clean up _stream if it exists and don't add anything new to it either. + _reqFinished = true; + if (_stream != nullptr) { + _stream->clearMsgs(); + } + } + + // This will clear the cyclic dependency: + // FileChannelShared -> ChannelStream -> SsiRequest -> FileChannelShared + // + // TODO: Eliminate xrdsvc::ChannelStream sinve this class seems to be useless + // in the file-based result delivery protocol. + _channelShared.reset(); + + auto keepAlive = freeSelfKeepAlive(); + + // No buffers allocated, so don't need to free. + // We can release/unlink the file now + const char* type = ""; + switch (rinfo.rType) { + case XrdSsiRespInfo::isNone: + type = "type=isNone"; + break; + case XrdSsiRespInfo::isData: + type = "type=isData"; + break; + case XrdSsiRespInfo::isError: + type = "type=isError"; + break; + case XrdSsiRespInfo::isFile: + type = "type=isFile"; + break; + case XrdSsiRespInfo::isStream: + type = "type=isStream"; + break; + case XrdSsiRespInfo::isHandle: + type = "type=isHandle"; + break; + } + + // Decrement the counter of the database/chunk resources in use + ResourceUnit ru(_resourceName); + if (ru.unitType() == ResourceUnit::DBCHUNK) { + _foreman->resourceMonitor()->decrement(_resourceName); + } + + // We can't do much other than close the file. + // It should work (on linux) to unlink the file after we open it, though. + // With the optimizer on '-Og', there was a double free for a SsiRequest. + // The likely cause could be keepAlive being optimized out for being unused. + // The problem has not reoccurred since adding keepAlive to the following + // comment, but having code depend on a comment line is ugly in its own way. + LOGS(_log, LOG_LVL_DEBUG, "RequestFinished " << type << " " << keepAlive.use_count()); +} + +bool SsiRequest::reply(char const* buf, int bufLen) { + Status s = SetResponse(buf, bufLen); + if (s != XrdSsiResponder::wasPosted) { + LOGS(_log, LOG_LVL_ERROR, "DANGER: Couldn't post response of length=" << bufLen); + return false; + } + return true; +} + +bool SsiRequest::replyError(std::string const& msg, int code) { + Status s = SetErrResponse(msg.c_str(), code); + if (s != XrdSsiResponder::wasPosted) { + LOGS(_log, LOG_LVL_ERROR, "DANGER: Couldn't post error response " << msg); + return false; + } + return true; +} + +bool SsiRequest::replyStream(StreamBuffer::Ptr const& sBuf, bool last) { + LOGS(_log, LOG_LVL_DEBUG, "replyStream, checking stream size=" << sBuf->getSize() << " last=" << last); + + // Normally, XrdSsi would call Recycle() when it is done with sBuf, but if this function + // returns false, then it must call Recycle(). Otherwise, the scheduler will likely + // wedge waiting for the buffer to be released. + std::lock_guard finLock(_finMutex); + if (_reqFinished) { + // Finished() was called, give up. + LOGS(_log, LOG_LVL_ERROR, "replyStream called after reqFinished."); + sBuf->Recycle(); + return false; + } + // Create a stream if needed. + if (!_stream) { + _stream = std::make_shared(); + if (SetResponse(_stream.get()) != XrdSsiResponder::Status::wasPosted) { + LOGS(_log, LOG_LVL_WARN, "SetResponse stream failed, calling Recycle for sBuf"); + // SetResponse return value indicates XrdSsi wont call Recycle(). + sBuf->Recycle(); + return false; + } + } else if (_stream->closed()) { + // XrdSsi isn't going to call Recycle if we wind up here. + LOGS(_log, LOG_LVL_ERROR, "Logic error SsiRequest::replyStream called with stream closed."); + sBuf->Recycle(); + return false; + } + // XrdSsi or Finished() will call Recycle(). + LOGS(_log, LOG_LVL_INFO, "SsiRequest::replyStream seq=" << getSeq()); + _stream->append(sBuf, last); + return true; +} + +bool SsiRequest::sendMetadata(const char* buf, int blen) { + Status stat = SetMetadata(buf, blen); + switch (stat) { + case XrdSsiResponder::wasPosted: + return true; + case XrdSsiResponder::notActive: + LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " notActive"); + break; + case XrdSsiResponder::notPosted: + LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " notPosted blen=" << blen); + break; + default: + LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " unkown state blen=" << blen); + } + return false; +} + +SsiRequest::Ptr SsiRequest::freeSelfKeepAlive() { + Ptr keepAlive = std::move(_selfKeepAlive); + return keepAlive; +} + +uint64_t SsiRequest::getSeq() const { + if (_stream == nullptr) return 0; + return _stream->getSeq(); +} + +} // namespace lsst::qserv::xrdsvc From 4b0b3df2c44302ca7b18c136bbd74f930d8c398e Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 16 May 2024 11:22:18 -0700 Subject: [PATCH 05/15] Czar and workers can send http messages to each other. --- src/ccontrol/MergingHandler.cc | 175 +++++++++++++++ src/ccontrol/MergingHandler.h | 11 + src/ccontrol/UserQueryFactory.cc | 3 - src/ccontrol/UserQuerySelect.cc | 167 +++++++++++++++ src/ccontrol/UserQuerySelect.h | 1 - src/czar/ActiveWorker.cc | 308 +++++++++++++++++++++++++++ src/czar/Czar.cc | 15 +- src/czar/CzarChunkMap.cc | 5 +- src/czar/CzarChunkMap.h | 2 - src/czar/CzarRegistry.cc | 9 - src/czar/CzarRegistry.h | 5 +- src/czar/HttpCzarWorkerModule.cc | 79 +++++++ src/czar/HttpCzarWorkerModule.h | 5 + src/czar/testCzar.cc | 8 +- src/http/CMakeLists.txt | 2 + src/http/Module.h | 2 + src/http/WorkerQueryStatusData.cc | 331 +++++++++++++++++++++++++++++ src/http/WorkerQueryStatusData.h | 234 ++++++++++++++++++++ src/http/testStatusData.cc | 140 ++++++++++++ src/proto/CMakeLists.txt | 1 + src/proto/worker.proto | 1 + src/protojson/ScanTableInfo.h | 2 +- src/qdisp/Executive.cc | 123 ++++++----- src/qdisp/Executive.h | 42 ++-- src/qdisp/JobBase.cc | 54 +++++ src/qdisp/JobBase.h | 78 +++++++ src/qdisp/JobDescription.cc | 1 - src/qdisp/JobDescription.h | 4 +- src/qdisp/JobQuery.cc | 20 +- src/qdisp/JobQuery.h | 13 +- src/qdisp/ResponseHandler.h | 11 + src/qdisp/UberJob.cc | 231 ++++++++++++++++++++ src/qdisp/UberJob.h | 1 - src/qmeta/CMakeLists.txt | 8 - src/qmeta/QMeta.h | 44 ---- src/qproc/TaskMsgFactory.h | 75 +++++++ src/rproc/InfileMerger.cc | 111 ++++++++++ src/rproc/InfileMerger.h | 3 + src/wbase/MsgProcessor.h | 64 ++++++ src/wbase/Task.cc | 106 ++++++++- src/wbase/UberJobData.cc | 11 +- src/wbase/UberJobData.h | 15 ++ src/wdb/QueryRunner.cc | 1 + src/wdb/testQueryRunner.cc | 154 ++++++++++++++ src/wdb/testQuerySql.cc | 101 +++++++++ src/wsched/ChunkTasksQueue.cc | 2 +- src/wsched/testSchedulers.cc | 1 + src/xrdreq/QueryManagementAction.h | 97 +++++++++ src/xrdsvc/ChannelStream.h | 75 +++++++ src/xrdsvc/HttpSvc.cc | 6 + src/xrdsvc/SsiRequest.cc | 84 +------- src/xrdsvc/SsiRequest.h | 128 +++++++++++ 52 files changed, 2929 insertions(+), 241 deletions(-) create mode 100644 src/http/WorkerQueryStatusData.cc create mode 100644 src/http/WorkerQueryStatusData.h create mode 100644 src/http/testStatusData.cc create mode 100644 src/qdisp/JobBase.cc create mode 100644 src/qdisp/JobBase.h create mode 100644 src/qproc/TaskMsgFactory.h create mode 100644 src/wbase/MsgProcessor.h create mode 100644 src/wdb/testQuerySql.cc create mode 100644 src/xrdreq/QueryManagementAction.h create mode 100644 src/xrdsvc/ChannelStream.h create mode 100644 src/xrdsvc/SsiRequest.h diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 18e2785dce..20ee860cac 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -259,6 +259,166 @@ std::tuple readHttpFileAndMergeHttp( return {success, mergeHappened}; } +std::tuple readHttpFileAndMergeHttp( + lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, + function const& messageIsReady, + shared_ptr const& httpConnPool) { + string const context = "MergingHandler::" + string(__func__) + " " + " qid=" + uberJob->getIdStr() + " "; + + LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); + + // Track the file while the control flow is staying within the function. + ResultFileTracker const resultFileTracker; + + // The data transmit rate tracker is set up before reading each data message. + unique_ptr> transmitRateTracker; + + // A location of the next byte to be read from the input file. The variable + // is used for error reporting. + uint64_t offset = 0; + + // Temporary buffer for messages read from the file. The buffer gets automatically + // resized to fit the largest message. + unique_ptr msgBuf; + size_t msgBufSize = 0; + size_t msgBufNext = 0; // An index of the next character in the buffer. + + // Fixed-size buffer to store the message size. + string msgSizeBuf(sizeof(uint32_t), '\0'); + size_t msgSizeBufNext = 0; // An index of the next character in the buffer. + + // The size of the next/current message. The variable is set after succesfully parsing + // the message length header and is reset back to 0 after parsing the message body. + // The value is stays 0 while reading the frame header. + uint32_t msgSizeBytes = 0; + bool success = true; + bool mergeSuccess = true; + int headerCount = 0; + uint64_t totalBytesRead = 0; + try { + string const noClientData; + vector const noClientHeaders; + http::ClientConfig clientConfig; + clientConfig.httpVersion = CURL_HTTP_VERSION_1_1; // same as in qhttp + clientConfig.bufferSize = CURL_MAX_READ_SIZE; // 10 MB in the current version of libcurl + clientConfig.tcpKeepAlive = true; + clientConfig.tcpKeepIdle = 5; // the default is 60 sec + clientConfig.tcpKeepIntvl = 5; // the default is 60 sec + http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, + httpConnPool); + reader.read([&](char const* inBuf, size_t inBufSize) { + // A value of the flag is set by the message processor when it's time to finish + // or abort reading the file. + bool last = false; + char const* next = inBuf; + char const* const end = inBuf + inBufSize; + while ((next < end) && !last) { + LOGS(_log, LOG_LVL_WARN, + context << "TODO:UJ next=" << (uint64_t)next << " end=" << (uint64_t)end + << " last=" << last); + if (msgSizeBytes == 0) { + // Continue or finish reading the frame header. + size_t const bytes2read = + std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); + std::memcpy(msgSizeBuf.data() + msgSizeBufNext, next, bytes2read); + next += bytes2read; + offset += bytes2read; + msgSizeBufNext += bytes2read; + if (msgSizeBufNext == sizeof(uint32_t)) { + ++headerCount; + // Done reading the frame header. + msgSizeBufNext = 0; + // Parse and evaluate the message length. + msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); + if (msgSizeBytes == 0) { + throw runtime_error("message size is 0 at offset " + + to_string(offset - sizeof(uint32_t)) + ", file: " + httpUrl); + } + if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { + throw runtime_error("message size " + to_string(msgSizeBytes) + " at offset " + + to_string(offset - sizeof(uint32_t)) + + " exceeds the hard limit of " + + to_string(ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) + + ", file: " + httpUrl); + } + // Extend the message buffer (if needed). Note that buffer never gets + // truncated to avoid excessive memory deallocations/allocations. + if (msgBufSize < msgSizeBytes) { + msgBufSize = msgSizeBytes; + msgBuf.reset(new char[msgBufSize]); + } + // Starts the tracker to measure the performance of the network I/O. + transmitRateTracker = + make_unique>(reportFileRecvRate); + } + } else { + // Continue or finish reading the message body. + size_t const bytes2read = + std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); + std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); + next += bytes2read; + offset += bytes2read; + msgBufNext += bytes2read; + if (msgBufNext == msgSizeBytes) { + // Done reading message body. + msgBufNext = 0; + + // Destroying the tracker will result in stopping the tracker's timer and + // reporting the file read rate before proceeding to the merge. + if (transmitRateTracker != nullptr) { + transmitRateTracker->addToValue(msgSizeBytes); + transmitRateTracker->setSuccess(); + transmitRateTracker.reset(); + } + + // Parse and evaluate the message. + mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); + totalBytesRead += msgSizeBytes; + if (!mergeSuccess) { + success = false; + throw runtime_error("message processing failed at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + // Reset the variable to prepare for reading the next header & message (if any). + msgSizeBytes = 0; + } else { + LOGS(_log, LOG_LVL_WARN, + context << " headerCount=" << headerCount + << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); + } + } + } + }); + LOGS(_log, LOG_LVL_DEBUG, + context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes + << " totalBytesRead=" << totalBytesRead); + if (msgSizeBufNext != 0) { + throw runtime_error("short read of the message header at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + if (msgBufNext != 0) { + throw runtime_error("short read of the message body at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); + success = false; + } + + // Remove the file from the worker if it still exists. Report and ignore errors. + // The files will be garbage-collected by workers. + try { + http::Client remover(http::Method::DELETE, httpUrl); + remover.read([](char const* inBuf, size_t inBufSize) {}); + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, context << "failed to remove " << httpUrl << ", ex: " << ex.what()); + } + // If the merge failed, that indicates something went wrong in the local database table, + // is likely this user query is doomed and should be cancelled. + LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeSuccess); + return {success, mergeSuccess}; +} + } // namespace namespace lsst::qserv::ccontrol { @@ -266,6 +426,7 @@ namespace lsst::qserv::ccontrol { MergingHandler::MergingHandler(std::shared_ptr merger, std::string const& tableName) : _infileMerger{merger}, _tableName{tableName} {} + MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_TRACE, __func__ << " " << _tableName); } @@ -345,6 +506,20 @@ bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, return success; } +bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, + proto::ResponseData const& responseData) { + if (_flushed) { + throw util::Bug(ERR_LOC, "already flushed"); + } + bool const success = _infileMerger->mergeHttp(uberJob, responseData); + if (!success) { + LOGS(_log, LOG_LVL_WARN, __func__ << " failed"); + util::Error const& err = _infileMerger->getError(); + _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg()); + } + return success; +} + void MergingHandler::_setError(int code, std::string const& msg) { LOGS(_log, LOG_LVL_DEBUG, "_setError: code: " << code << ", message: " << msg); std::lock_guard lock(_errorMutex); diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index fc5b7e0d37..6876e9cd68 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -79,6 +79,14 @@ class MergingHandler : public qdisp::ResponseHandler { /// @see ResponseHandler::flushHttpError void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; + /// @see ResponseHandler::flushHttp + /// @see MerginHandler::_mergeHttp + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) override; + + /// @see ResponseHandler::flushHttpError + void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; + /// Signal an unrecoverable error condition. No further calls are expected. void errorFlush(std::string const& msg, int code) override; @@ -95,6 +103,9 @@ class MergingHandler : public qdisp::ResponseHandler { /// Call InfileMerger to do the work of merging this data to the result. bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + /// Call InfileMerger to do the work of merging this data to the result. + bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + /// Set error code and string. void _setError(int code, std::string const& msg); diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 2ced51d111..71843811e4 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -267,9 +267,6 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st // First check for SUBMIT and strip it std::string query = aQuery; - // TODO: DM-43386 need to have WorkerChunkMap info at this point - // &&& - std::string stripped; bool async = false; if (UserQueryType::isSubmit(query, stripped)) { diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index fa04c8bc6f..beaec52717 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -227,6 +227,7 @@ std::string UserQuerySelect::getResultQuery() const { return resultQuery; } +#if 0 // &&& /// Begin running on all chunks added so far. void UserQuerySelect::submit() { auto exec = _executive; @@ -322,6 +323,7 @@ void UserQuerySelect::submit() { _qMetaAddChunks(chunks); } } +#endif //&&& void UserQuerySelect::buildAndSendUberJobs() { // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? @@ -467,6 +469,21 @@ void UserQuerySelect::buildAndSendUberJobs() { } wInfUJ->wInf = iter->second; } + // Add this job to the appropriate UberJob, making the UberJob if needed. + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c5"); + string workerId = targetWorker->getWorkerId(); + auto& ujVect = workerJobMap[workerId]; + if (ujVect.empty() || ujVect.back()->getJobCount() > maxChunksPerUber) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c5a"); + string uberResultName = ttn.make(uberJobId); + auto respHandler = make_shared(_infileMerger, uberResultName); + auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), uberJobId++, + _qMetaCzarId, targetWorker); + ujVect.push_back(uJob); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c6"); + ujVect.back()->addJob(jqPtr); + } if (wInfUJ->uberJobPtr == nullptr) { auto ujId = _uberJobIdSeq++; // keep ujId consistent @@ -511,6 +528,156 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_DEBUG, funcN << " " << exec->dumpUberJobCounts()); } +void UserQuerySelect::buildAndSendUberJobs() { + string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start"); + + // Ensure `_monitor()` doesn't do anything until everything is ready. + if (!_executive->isReadyToExecute()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive isn't ready to generate UberJobs."); + return; + } + + // Only one thread should be generating UberJobs for this user query at any given time. + lock_guard fcLock(_buildUberJobMtx); + LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << _executive->getTotalJobs()); + + vector uberJobs; + + qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); + if (unassignedChunksInQuery.empty()) { + LOGS(_log, LOG_LVL_TRACE, funcN << " no unassigned Jobs"); + return; + } + + auto czarPtr = czar::Czar::getCzar(); + auto czFamilyMap = czarPtr->getCzarFamilyMap(); + auto czChunkMap = czFamilyMap->getChunkMap(_queryDbName); + auto czRegistry = czarPtr->getCzarRegistry(); + + if (czChunkMap == nullptr) { + LOGS(_log, LOG_LVL_ERROR, funcN << " no map found for queryDbName=" << _queryDbName); + // Make an empty chunk map so all jobs are flagged as needing to be reassigned. + // There's a chance that a family will be replicated by the registry. + czChunkMap = czar::CzarChunkMap::create(); + } + + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); + // Make a map of all jobs in the executive. + // TODO:UJ Maybe a check should be made that all databases are in the same family? + + + + // keep cycling through workers until no more chunks to place. + // - create a map of UberJobs key=, val=> + // - for chunkId in `unassignedChunksInQuery` + // - use `chunkMapPtr` to find the shared scan workerId for chunkId + // - if not existing in the map, make a new uberjob + // - if existing uberjob at max jobs, create a new uberjob + // - once all chunks in the query have been put in uberjobs, find contact info + // for each worker + // - add worker to each uberjob. + // - For failures - If a worker cannot be contacted, that's an uberjob failure. + // - uberjob failures (due to communications problems) will result in the uberjob + // being broken up into multiple UberJobs going to different workers. + // - The best way to do this is probably to just kill the UberJob and mark all + // Jobs that were in that UberJob as needing re-assignment, and re-running + // the code here. The trick is going to be figuring out which workers are alive. + // Maybe force a fresh lookup from the replicator Registry when an UberJob fails. + map> workerJobMap; + vector missingChunks; + + // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in + // numerical order. The workers run shared scans in numerical order of chunk id numbers. + // This keeps the number of partially complete UberJobs running on a worker to a minimum, + // and should minimize the time for the first UberJob on the worker to complete. + for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { + auto iter = chunkMapPtr->find(chunkId); + if (iter == chunkMapPtr->end()) { + missingChunks.push_back(chunkId); + bool const increaseAttemptCount = true; + jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); + // Assign as many jobs as possible. Any chunks not found will be attempted later. + continue; + } + czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; + auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + // TODO:UJ maybe if (targetWorker == nullptr || this worker already tried for this chunk) { + if (targetWorker == nullptr) { + LOGS(_log, LOG_LVL_ERROR, funcN << " No primary scan worker for chunk=" << chunkData->dump()); + // Try to assign a different worker to this job + auto workerHasThisChunkMap = chunkData->getWorkerHasThisMapCopy(); + bool found = false; + for (auto wIter = workerHasThisChunkMap.begin(); wIter != workerHasThisChunkMap.end() && !found; + ++wIter) { + auto maybeTarg = wIter->second.lock(); + if (maybeTarg != nullptr) { + targetWorker = maybeTarg; + found = true; + LOGS(_log, LOG_LVL_WARN, + funcN << " Alternate worker found for chunk=" << chunkData->dump()); + } + } + if (!found) { + // If too many workers are down, there will be a chunk that cannot be found. + // Just continuing should leave jobs `unassigned` with their attempt count + // increased. Either the chunk will be found and jobs assigned, or the jobs' + // attempt count will reach max and the query will be cancelled + // TODO:UJ Needs testing/verification + LOGS(_log, LOG_LVL_ERROR, + funcN << " No primary or alternate worker found for chunk=" << chunkData->dump()); + continue; + } + } + // Add this job to the appropriate UberJob, making the UberJob if needed. + string workerId = targetWorker->getWorkerId(); + auto& ujVect = workerJobMap[workerId]; + if (ujVect.empty() || ujVect.back()->getJobCount() >= _maxChunksPerUberJob) { + auto ujId = _uberJobIdSeq++; // keep ujId consistent + string uberResultName = _ttn->make(ujId); + auto respHandler = make_shared(_infileMerger, uberResultName); + auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), ujId, + _qMetaCzarId, targetWorker); + ujVect.push_back(uJob); + } + auto& ujVectBack = ujVect.back(); + ujVectBack->addJob(jqPtr); + LOGS(_log, LOG_LVL_DEBUG, + funcN << " ujVectBack{" << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() + << "}"); + } + + if (!missingChunks.empty()) { + string errStr = funcN + " a worker could not be found for these chunks "; + for (auto const& chk : missingChunks) { + errStr += to_string(chk) + ","; + } + errStr += " they will be retried later."; + LOGS(_log, LOG_LVL_ERROR, errStr); + } + + // Add worker contact info to UberJobs. + auto const wContactMap = czRegistry->getWorkerContactMap(); + LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); + for (auto const& [wIdKey, ujVect] : workerJobMap) { + auto iter = wContactMap->find(wIdKey); + if (iter == wContactMap->end()) { + // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. + // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, + // so maybe return internal error to the user? + throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + wIdKey); + } + auto const& wContactInfo = iter->second; + for (auto const& ujPtr : ujVect) { + ujPtr->setWorkerContactInfo(wContactInfo); + } + _executive->addUberJobs(ujVect); + for (auto const& ujPtr : ujVect) { + _executive->runUberJob(ujPtr); + } + } +} + /// Block until a submit()'ed query completes. /// @return the QueryState indicating success or failure QueryState UserQuerySelect::join() { diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index 11c50e70fc..6447f8fcd5 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -110,7 +110,6 @@ class UserQuerySelect : public UserQuery { /// Begin execution of the query over all ChunkSpecs added so far. void submit() override; - void submitOld(); //&&& /// Wait until the query has completed execution. /// @return the final execution state. diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index 8508fec0a4..b6454cd83a 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -236,6 +236,314 @@ string ActiveWorker::_dump() const { return os.str(); } +string ActiveWorker::getStateStr(State st) { + switch (st) { + case ALIVE: return string("ALIVE"); + case QUESTIONABLE: return string("QUESTIONABLE"); + case DEAD: return string("DEAD"); + } + return string("unknown"); +} + +bool ActiveWorker::compareContactInfo(http::WorkerContactInfo const& wcInfo) const { + lock_guard lg(_aMtx); + return _wqsData->_wInfo->isSameContactInfo(wcInfo); +} + +void ActiveWorker::setWorkerContactInfo(http::WorkerContactInfo::Ptr const& wcInfo) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " new info=" << wcInfo->dump()); + lock_guard lg(_aMtx); + _wqsData->_wInfo = wcInfo; +} + +void ActiveWorker::_changeStateTo(State newState, double secsSinceUpdate, string const& note) { + auto lLvl = (newState == DEAD) ? LOG_LVL_ERROR : LOG_LVL_INFO; + LOGS(_log, lLvl, note << " oldState=" << getStateStr(_state) << " newState=" << getStateStr(newState) << " secsSince=" << secsSinceUpdate); + _state = newState; +} + +void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime) { + // &&& function too long + lock_guard lg(_aMtx); + double secsSinceUpdate = _wqsData->_wInfo->timeSinceRegUpdateSeconds(); + // Update the last time the registry contacted this worker. + switch (_state) { + case ALIVE: { + if (secsSinceUpdate > timeoutAliveSecs) { + _changeStateTo(QUESTIONABLE, secsSinceUpdate, cName(__func__)); + // Anything that should be done here? + } + break; + } + case QUESTIONABLE: { + if (secsSinceUpdate < timeoutAliveSecs) { + _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); + } + if (secsSinceUpdate > timeoutDeadSecs) { + _changeStateTo(DEAD, secsSinceUpdate, cName(__func__)); + // &&& TODO:UJ all uberjobs for this worker need to die. + } + break; + } + case DEAD: { + LOGS(_log, LOG_LVL_ERROR, "&&& NEED CODE"); + if (secsSinceUpdate < timeoutAliveSecs) { + _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); + } else { + // Don't waste time on this worker until the registry has heard from it. + return; + } + break; + } + + } + + // Check how many messages are currently being sent to the worker, if at the limit, return + if (_wqsData->_qIdDoneKeepFiles.empty() && _wqsData->_qIdDoneDeleteFiles.empty() && _wqsData->_qIdDeadUberJobs.empty()) { + return; + } + int tCount = _conThreadCount; + if (tCount > _maxConThreadCount) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " not sending message since at max threads " << tCount); + return; + } + + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // message to send to the worker. +#if 0 // &&& + auto now = CLOCK::now(); + auto const czarConfig = cconfig::CzarConfig::instance(); + + shared_ptr jsWorkerReqPtr = make_shared(); + json& jsWorkerR = *jsWorkerReqPtr; + jsWorkerR["version"] = http::MetaModule::version; + jsWorkerR["instance_id"] = czarConfig->replicationInstanceId(); + jsWorkerR["auth_key"] = czarConfig->replicationAuthKey(); + jsWorkerR["worker"] = _wInfo->wId; + jsWorkerR["qiddonekeepfiles"] = json::array(); + jsWorkerR["qiddonedeletefiles"] = json::array(); + jsWorkerR["qiddeaduberjobs"] = json::array(); + jsWorkerR["czar"] = json::object(); + auto& jsWCzar = jsWorkerR["czar"]; + jsWCzar["name"] = czarConfig->name(); + jsWCzar["id"]= czarConfig->id(); + jsWCzar["management-port"] = czarConfig->replicationHttpPort(); + jsWCzar["management-host-name"] = util::get_current_host_fqdn(); + + + { + auto& jsDoneKeep = jsWorkerR["qiddonekeepfiles"]; + auto iterDoneKeep = _qIdDoneKeepFiles.begin(); + while (iterDoneKeep != _qIdDoneKeepFiles.end()) { + auto qId = iterDoneKeep->first; + jsDoneKeep.push_back(qId); + auto tmStamp = iterDoneKeep->second; + double ageSecs = std::chrono::duration(now - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterDoneKeep = _qIdDoneKeepFiles.erase(iterDoneKeep); + } else { + ++iterDoneKeep; + } + } + } + { + auto& jsDoneDelete = jsWorkerR["qiddonedeletefiles"]; + auto iterDoneDelete = _qIdDoneDeleteFiles.begin(); + while (iterDoneDelete != _qIdDoneDeleteFiles.end()) { + auto qId = iterDoneDelete->first; + jsDoneDelete.push_back(qId); + auto tmStamp = iterDoneDelete->second; + double ageSecs = std::chrono::duration(now - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterDoneDelete = _qIdDoneDeleteFiles.erase(iterDoneDelete); + } else { + ++iterDoneDelete; + } + } + } + { + auto& jsDeadUj = jsWorkerR["qiddeaduberjobs"]; + auto iterDeadUjQid = _qIdDeadUberJobs.begin(); + while (iterDeadUjQid != _qIdDeadUberJobs.end()) { + TIMEPOINT oldestTm; // default is zero + auto qId = iterDeadUjQid->first; + auto& ujIdMap = iterDeadUjQid->second; + + json jsQidUj = {{"qid", qId}, {"ujids", json::array()}}; + auto& jsUjIds = jsQidUj["ujids"]; + + auto iterUjId = ujIdMap.begin(); + bool addedUjId = false; + while (iterUjId != ujIdMap.end()) { + UberJobId ujId = iterUjId->first; + auto tmStamp = iterUjId->second; + if (tmStamp > oldestTm) { + oldestTm = tmStamp; + } + + jsUjIds.push_back(ujId); + addedUjId = true; + double ageSecs = std::chrono::duration(now - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterUjId = ujIdMap.erase(iterUjId); + } else { + ++iterUjId; + } + } + + if (addedUjId) { + jsDeadUj.push_back(jsQidUj); + } + + if (ujIdMap.empty() + || std::chrono::duration(now - oldestTm).count() > maxLifetime) { + iterDeadUjQid = _qIdDeadUberJobs.erase(iterDeadUjQid); + } else { + ++iterDeadUjQid; + } + } + } +#endif // &&& + + auto jsWorkerReqPtr = _wqsData->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + + // Start a thread to send the message. (Maybe these should go on the qdisppool? &&&) + // put this in a different function and start the thread.&&&; + _sendStatusMsg(jsWorkerReqPtr); +} + +#if 0 // &&& +bool ActiveWorker::_parse(nlohmann::json const& jsWorkerReq) { + auto const czarConfig = cconfig::CzarConfig::instance(); + + http::RequestBodyJSON rbWReq(jsWorkerReq); + if (jsWorkerReq["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " bad version"); + return false; + } + + + http::RequestBodyJSON rbCzar(rbWReq.required("czar")); + auto czarName = rbCzar.required("name"); + auto czarId = rbCzar.required("id"); + auto czarPort = rbCzar.required("management-port"); + auto czarHostName = rbCzar.required("management-host-name"); + /* &&& + jsWorkerReq["instance_id"] != czarConfig->replicationInstanceId(); + jsWorkerReq["auth_key"] != czarConfig->replicationAuthKey(); + jsWorkerReq["worker"] != _wInfo->wId; + auto& jsWCzar = jsWorkerReq["czar"]; + jsWCzar["name"] != czarConfig->name(); + jsWCzar["id"] != czarConfig->id(); + jsWCzar["management-port"] != czarConfig->replicationHttpPort(); + jsWCzar["management-host-name"] != util::get_current_host_fqdn(); + */ + + + auto& jsQIdDoneKeepFiles = jsWorkerReq["qiddonekeepfiles"]; + for (auto const& qidKeep : jsQIdDoneKeepFiles) { + + } + + auto& jsQIdDoneDeleteFiles = jsWorkerReq["qiddonedeletefiles"]; + + auto& jsQIdDeadUberJobs = jsWorkerReq["qiddeaduberjobs"]; + +} +#endif // &&& + +void ActiveWorker::_sendStatusMsg(std::shared_ptr const& jsWorkerReqPtr) { + + auto& jsWorkerReq = *jsWorkerReqPtr; + auto const method = http::Method::POST; + auto const& wInf = _wqsData->_wInfo; + string const url = "http://" + wInf->wHost + ":" + to_string(wInf->wPort) + "/querystatus"; + vector const headers = {"Content-Type: application/json"}; + auto const& czarConfig = cconfig::CzarConfig::instance(); + + + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << jsWorkerReq); + string const requestContext = "Czar: '" + http::method2string(method) + "' stat request to '" + url + "'"; + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << " czarPost url=" << url << " request=" << jsWorkerReq.dump() + << " headers=" << headers[0]); + http::Client client(method, url, jsWorkerReq.dump(), headers); + bool transmitSuccess = false; + string exceptionWhat; + try { + json const response = client.readAsJson(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " response success=0"); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + exceptionWhat = ex.what(); + } + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure"); + } else { + // parse the return statement and remove the indicated entries from the list + //HERE &&&; + } +} + + +string ActiveWorker::dump() const { + lock_guard lg(_aMtx); + return _dump(); +} + +string ActiveWorker::_dump() const { + stringstream os; + os << "ActiveWorker " << (_wqsData->dump()); + return os.str(); +} + + +void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, http::CzarContactInfo::Ptr const& czInfo, std::string const& replicationInstanceId, std::string const& replicationAuthKey) { + // Go through wcMap, update existing entries in _awMap, create new entries for those that don't exist, + lock_guard awLg(_awMapMtx); + for (auto const& [wcKey, wcVal] : wcMap) { + auto iter = _awMap.find(wcKey); + if (iter == _awMap.end()) { + auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); + _awMap[wcKey] = newAW; + } else { + auto aWorker = iter->second; + if (!aWorker->compareContactInfo(*wcVal)) { + // This should not happen, but try to handle it gracefully if it does. + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " worker contact info changed for " << wcKey << " new=" << wcVal->dump() << " old=" << aWorker->dump()); + aWorker->setWorkerContactInfo(wcVal); + } + } + } +} + +/* &&& +void ActiveWorkerMap::pruneMap() { + lock_guard awLg(_awMapMtx); + for (auto iter = _awMap.begin(); iter != _awMap.end();) { + auto aWorker = iter->second; + if (aWorker->getWInfo()->timeSinceTouchSeconds() > _maxDeadTimeSeconds) { + iter = _awMap.erase(iter); + } else { + ++iter; + } + } +} +*/ + +void ActiveWorkerMap::sendActiveWorkersMessages() { + // Send messages to each active worker as needed + lock_guard lck(_awMapMtx); + for(auto&& [wName, awPtr] : _awMap) { + awPtr->updateStateAndSendMessages(_timeoutAliveSecs, _timeoutDeadSecs, _maxLifetime); + } +} + + ActiveWorkerMap::ActiveWorkerMap(std::shared_ptr const& czarConfig) : _timeoutAliveSecs(czarConfig->getActiveWorkerTimeoutAliveSecs()), _timeoutDeadSecs(czarConfig->getActiveWorkerTimeoutDeadSecs()), diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index cb118f676e..727ebafeed 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -71,6 +71,7 @@ using namespace std; namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.Czar"); } // anonymous namespace @@ -112,6 +113,9 @@ void Czar::_monitor() { // check if workers have died by timeout. _czarRegistry->sendActiveWorkersMessages(); + // &&& Send appropriate messages to all ActiveWorkers + _czarRegistry->sendActiveWorkersMessages(); + /// Create new UberJobs (if possible) for all jobs that are /// unassigned for any reason. map> execMap; @@ -173,11 +177,8 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->czarId); - try { - _czarChunkMap = CzarChunkMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); - } catch (ChunkMapException const& exc) { - LOGS(_log, LOG_LVL_WARN, string(__func__) + " failed to create CzarChunkMap " + exc.what()); - } + // This will block until there is a successful read of the database tables. + _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); // Tell workers to cancel any queries that were submitted before this restart of Czar. // Figure out which query (if any) was recorded in Czar databases before the restart. @@ -262,7 +263,9 @@ Czar::~Czar() { Czar::~Czar() { LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar()"); - cout << "&&& Czar::~Czar()" << endl; + _monitorLoop = false; + _monitorThrd.join(); + LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar() end"); } SubmitResult Czar::submitQuery(string const& query, map const& hints) { diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index f16f82092b..7061d7ba09 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -25,7 +25,6 @@ // System headers #include -#include // &&& del // LSST headers #include "lsst/log/Log.h" @@ -289,7 +288,6 @@ string CzarChunkMap::WorkerChunksData::dump() const { return os.str(); } - CzarFamilyMap::Ptr CzarFamilyMap::create(std::shared_ptr const& qmeta) { // There's nothing the czar can do until with user queries until there's been at least // one successful read of the database family tables, as the czar doesn't know where to find anything. @@ -344,6 +342,9 @@ bool CzarFamilyMap::_read() { return false; } + // &&& TODO:UJ Before makeNewMaps(), get a list of workers considered to be alive by czar::_activeWorkerMap + // give that list to makeNewMaps, and don't and workers to the maps that aren't on the list.&&& !!! + // Make the new maps. auto czConfig = cconfig::CzarConfig::instance(); bool usingChunkSize = czConfig->getFamilyMapUsingChunkSize(); diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index 800b5671a8..8188dfa697 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -34,7 +34,6 @@ // Qserv headers #include "global/clock_defs.h" -// #include "qmeta/QMeta.h" &&& #include "util/Issue.h" namespace lsst::qserv::qmeta { @@ -52,7 +51,6 @@ class ChunkMapException : public util::Issue { ChunkMapException(Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} }; - /// This class is used to organize worker chunk table information so that it /// can be used to send jobs to the appropriate worker and inform workers /// what chunks they can expect to handle in shared scans. diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index 34826a0ecd..ac65fe5be1 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -61,18 +61,13 @@ CzarRegistry::CzarRegistry(cconfig::CzarConfig::Ptr const& czarConfig, } CzarRegistry::~CzarRegistry() { - cout << "&&& CzarRegistry::~CzarRegistry a" << endl; _loop = false; if (_czarHeartbeatThrd.joinable()) { - cout << "&&& CzarRegistry::~CzarRegistry a1" << endl; _czarHeartbeatThrd.join(); } - cout << "&&& CzarRegistry::~CzarRegistry b" << endl; if (_czarWorkerInfoThrd.joinable()) { - cout << "&&& CzarRegistry::~CzarRegistry b1" << endl; _czarWorkerInfoThrd.join(); } - cout << "&&& CzarRegistry::~CzarRegistry end" << endl; } protojson::WorkerContactInfo::WCMapPtr CzarRegistry::getWorkerContactMap() const { @@ -82,7 +77,6 @@ protojson::WorkerContactInfo::WCMapPtr CzarRegistry::getWorkerContactMap() const void CzarRegistry::_registryUpdateLoop() { - cout << "&&& CzarRegistry::_registryUpdateLoop a" << endl; auto const method = http::Method::POST; string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + to_string(_czarConfig->replicationRegistryPort()) + "/czar"; @@ -113,11 +107,9 @@ void CzarRegistry::_registryUpdateLoop() { } this_thread::sleep_for(chrono::seconds(max(1U, _czarConfig->replicationRegistryHearbeatIvalSec()))); } - cout << "&&& CzarRegistry::_registryUpdateLoop end" << endl; } void CzarRegistry::_registryWorkerInfoLoop() { - cout << "&&& CzarRegistry::_registryWorkerInfoLoop a" << endl; // Get worker information from the registry string const replicationInstanceId = _czarConfig->replicationInstanceId(); string const replicationAuthKey = _czarConfig->replicationAuthKey(); @@ -160,7 +152,6 @@ void CzarRegistry::_registryWorkerInfoLoop() { } this_thread::sleep_for(chrono::seconds(15)); } - cout << "&&& CzarRegistry::_registryWorkerInfoLoop end" << endl; } protojson::WorkerContactInfo::WCMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index dd73e08133..10b0dbcd22 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -91,11 +91,14 @@ class CzarRegistry { /// Return _contactMap, the object that the returned pointer points to is /// constant and no attempts should be made to change it. - WorkerContactMapPtr getWorkerContactMap() { + http::WorkerContactInfo::WCMapPtr getWorkerContactMap() { std::lock_guard lockG(_mapMtx); return _contactMap; } + /// &&& doc + void sendActiveWorkersMessages(); + private: CzarRegistry() = delete; CzarRegistry(std::shared_ptr const& czarConfig, diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 9d134aead5..de6fd92e17 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -225,4 +225,83 @@ json HttpCzarWorkerModule::_handleWorkerCzarComIssue(string const& func) { return jsRet; } +json HttpCzarWorkerModule::_handleJobError(string const& func) { + // Metadata-only responses for the file-based protocol should not have any data + + // Parse and verify the json message and then kill the UberJob. + json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + string const targetWorkerId = body().required("workerid"); + string const czarName = body().required("czar"); + qmeta::CzarId const czarId = body().required("czarid"); + QueryId const queryId = body().required("queryid"); + UberJobId const uberJobId = body().required("uberjobid"); + int const errorCode = body().required("errorCode"); + string const errorMsg = body().required("errorMsg"); + + // Find UberJob + qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); + if (exec == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No executive for qid=") + + to_string(queryId) + " czar=" + to_string(czarId)); + } + qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); + if (uj == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No UberJob for qid=") + + to_string(queryId) + " ujId=" + to_string(uberJobId) + + " czar=" + to_string(czarId)); + } + + auto importRes = uj->workerError(errorCode, errorMsg); + jsRet = importRes; + + } catch (std::invalid_argument const& iaEx) { + LOGS(_log, LOG_LVL_ERROR, + "HttpCzarWorkerModule::_handleJobError received " << iaEx.what() << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; + } + return jsRet; +} + +json HttpCzarWorkerModule::_handleJobReady(string const& func) { + // Metadata-only responses for the file-based protocol should not have any data + + // Parse and verify the json message and then have the uberjob import the file. + json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + string const targetWorkerId = body().required("workerid"); + string const czarName = body().required("czar"); + qmeta::CzarId const czarId = body().required("czarid"); + QueryId const queryId = body().required("queryid"); + UberJobId const uberJobId = body().required("uberjobid"); + string const fileUrl = body().required("fileUrl"); + uint64_t const rowCount = body().required("rowCount"); + uint64_t const fileSize = body().required("fileSize"); + + // Find UberJob + qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); + if (exec == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + + to_string(queryId) + " czar=" + to_string(czarId)); + } + qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); + if (uj == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No UberJob for qid=") + + to_string(queryId) + " ujId=" + to_string(uberJobId) + + " czar=" + to_string(czarId)); + } + + auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); + jsRet = importRes; + + } catch (std::invalid_argument const& iaEx) { + LOGS(_log, LOG_LVL_ERROR, + "HttpCzarWorkerModule::_handleJobReady received " << iaEx.what() << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; + } + return jsRet; +} + } // namespace lsst::qserv::czar diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h index a6d21536c0..e849172b6c 100644 --- a/src/czar/HttpCzarWorkerModule.h +++ b/src/czar/HttpCzarWorkerModule.h @@ -81,6 +81,11 @@ class HttpCzarWorkerModule : public QhttpModule { /// Translates the issues and calls the Czar to take action. nlohmann::json _handleWorkerCzarComIssue(std::string const& func); + /// &&& doc + nlohmann::json _handleJobReady(std::string const& func); + + /// Translates the error and calls the Czar to take action. + nlohmann::json _handleJobError(std::string const& func); }; } // namespace lsst::qserv::czar diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index 3a3c761e36..aad9fdfd31 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -89,7 +89,6 @@ qmeta::QMetaChunkMap convertJsonToChunkMap(nlohmann::json const& jsChunks) { BOOST_AUTO_TEST_CASE(CzarChunkMap) { // Each chunk only occurs on one worker - cerr << "&&& a" << endl; string test1 = R"( { "ce1c1b79-e6fb-11ee-a46b-0242c0a80308": @@ -126,7 +125,6 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } } )"; - cerr << "&&& b " << test1 << endl; /// 3 workers, each containing all chunks. string test2 = R"( @@ -187,21 +185,19 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } } )"; - cerr << "&&& c" << endl; auto dbToFamily = make_shared(); czar::CzarFamilyMap czFamMap(dbToFamily); auto jsTest1 = nlohmann::json::parse(test1); qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); - auto familyMap = czFamMap.makeNewMaps(qChunkMap1, true); + auto familyMap = czFamMap.makeNewMaps(qChunkMap1); czar::CzarFamilyMap::verify(familyMap); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 1 passed"); - cerr << "&&& g" << endl; auto jsTest2 = nlohmann::json::parse(test2); qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); - auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2, true); + auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2); czar::CzarFamilyMap::verify(familyMap2); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 2 passed"); } diff --git a/src/http/CMakeLists.txt b/src/http/CMakeLists.txt index 454d4ab88f..61097f9f22 100644 --- a/src/http/CMakeLists.txt +++ b/src/http/CMakeLists.txt @@ -19,6 +19,7 @@ target_sources(http PRIVATE RequestBodyJSON.cc RequestQuery.cc Url.cc + WorkerQueryStatusData.cc ) target_link_libraries(http PUBLIC @@ -51,5 +52,6 @@ http_tests( testAsyncReq testRequestBodyJSON testRequestQuery + testStatusData testUrl ) diff --git a/src/http/Module.h b/src/http/Module.h index 6bd149fb16..2cc5e42d95 100644 --- a/src/http/Module.h +++ b/src/http/Module.h @@ -92,6 +92,8 @@ class Module : public BaseModule { */ virtual void sendResponse(std::string const& content, std::string const& contentType) = 0; + std::string authKey() const { return _authKey; } + private: /** * Pull the raw request body and translate it into a JSON object. diff --git a/src/http/WorkerQueryStatusData.cc b/src/http/WorkerQueryStatusData.cc new file mode 100644 index 0000000000..cd254f7c07 --- /dev/null +++ b/src/http/WorkerQueryStatusData.cc @@ -0,0 +1,331 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "http/WorkerQueryStatusData.h" + +// System headers +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "util/common.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.http.WorkerQueryStatusData"); +} // namespace + +namespace lsst::qserv::http { + +json CzarContactInfo::serializeJson() const { + json jsCzar; + jsCzar["name"] = czName; + jsCzar["id"]= czId; + jsCzar["management-port"] = czPort; + jsCzar["management-host-name"] = czHostName; + return jsCzar; +} + +CzarContactInfo::Ptr CzarContactInfo::createJson(nlohmann::json const& czJson) { + try { + auto czName_ = RequestBodyJSON::required(czJson, "name"); + auto czId_ = RequestBodyJSON::required(czJson, "id"); + auto czPort_ = RequestBodyJSON::required(czJson, "management-port"); + auto czHostName_ = RequestBodyJSON::required(czJson, "management-host-name"); + return create(czName_, czId_, czPort_, czHostName_); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("CzarContactInfo::createJson invalid ") << exc.what()); + } + return nullptr; +} + +std::string CzarContactInfo::dump() const { + stringstream os; + os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << czHostName; + return os.str(); +} + + + +json WorkerContactInfo::serializeJson() const { + json jsWorker; + jsWorker["id"]= wId; + jsWorker["host"] = wHost; + jsWorker["management-host-name"] = wManagementHost; + jsWorker["management-port"] = wPort; + return jsWorker; +} + +WorkerContactInfo::Ptr WorkerContactInfo::createJson(nlohmann::json const& wJson, TIMEPOINT updateTime_) { + LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& a"); + try { + auto wId_ = RequestBodyJSON::required(wJson, "id"); + LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& b"); + auto wHost_ = RequestBodyJSON::required(wJson, "host"); + LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& c"); + auto wManagementHost_ = RequestBodyJSON::required(wJson, "management-host-name"); + LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& d"); + auto wPort_ = RequestBodyJSON::required(wJson, "management-port"); + LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& e"); + return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); + } + return nullptr; +} + + + +string WorkerContactInfo::dump() const { + stringstream os; + os << "workerContactInfo{" + << "id=" << wId << " host=" << wHost << " mgHost=" << wManagementHost << " port=" << wPort << "}"; + return os.str(); +} + +/* &&& +string ActiveWorker::getStateStr(State st) { + switch (st) { + case ALIVE: return string("ALIVE"); + case QUESTIONABLE: return string("QUESTIONABLE"); + case DEAD: return string("DEAD"); + } + return string("unknown"); +} + + +bool WorkerQueryStatusData::compareContactInfo(WorkerContactInfo const& wcInfo) const { + return _wInfo->isSameContactInfo(wcInfo); +} + +void WorkerQueryStatusData::setWorkerContactInfo(WorkerContactInfo::Ptr const& wcInfo) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " new info=" << wcInfo->dump()); + _wInfo = wcInfo; +} +*/ + + +shared_ptr WorkerQueryStatusData::serializeJson(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime) { + + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // message to send to the worker. + auto now = CLOCK::now(); + //&&&auto const czarConfig = cconfig::CzarConfig::instance(); + + shared_ptr jsWorkerReqPtr = make_shared(); + json& jsWorkerR = *jsWorkerReqPtr; + jsWorkerR["version"] = http::MetaModule::version; + /* &&& + jsWorkerR["instance_id"] = czarConfig->replicationInstanceId(); + jsWorkerR["auth_key"] = czarConfig->replicationAuthKey(); + */ + jsWorkerR["instance_id"] = _replicationInstanceId; + jsWorkerR["auth_key"] = _replicationAuthKey; + //&&&jsWorkerR["worker"] = _wInfo->wId; + jsWorkerR["qiddonekeepfiles"] = json::array(); + jsWorkerR["qiddonedeletefiles"] = json::array(); + jsWorkerR["qiddeaduberjobs"] = json::array(); + //&&&jsWorkerR["czar"] = json::object(); + jsWorkerR["czar"] = _czInfo->serializeJson(); + //&&&jsWorkerR["worker"] = json::object(); + jsWorkerR["worker"] = _wInfo->serializeJson(); + + + { + auto& jsDoneKeep = jsWorkerR["qiddonekeepfiles"]; + auto iterDoneKeep = _qIdDoneKeepFiles.begin(); + while (iterDoneKeep != _qIdDoneKeepFiles.end()) { + auto qId = iterDoneKeep->first; + jsDoneKeep.push_back(qId); + auto tmStamp = iterDoneKeep->second; + double ageSecs = std::chrono::duration(now - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterDoneKeep = _qIdDoneKeepFiles.erase(iterDoneKeep); + } else { + ++iterDoneKeep; + } + } + } + { + auto& jsDoneDelete = jsWorkerR["qiddonedeletefiles"]; + auto iterDoneDelete = _qIdDoneDeleteFiles.begin(); + while (iterDoneDelete != _qIdDoneDeleteFiles.end()) { + auto qId = iterDoneDelete->first; + jsDoneDelete.push_back(qId); + auto tmStamp = iterDoneDelete->second; + double ageSecs = std::chrono::duration(now - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterDoneDelete = _qIdDoneDeleteFiles.erase(iterDoneDelete); + } else { + ++iterDoneDelete; + } + } + } + { + auto& jsDeadUj = jsWorkerR["qiddeaduberjobs"]; + auto iterDeadUjQid = _qIdDeadUberJobs.begin(); + while (iterDeadUjQid != _qIdDeadUberJobs.end()) { + TIMEPOINT oldestTm; // default is zero + auto qId = iterDeadUjQid->first; + auto& ujIdMap = iterDeadUjQid->second; + + json jsQidUj = {{"qid", qId}, {"ujids", json::array()}}; + auto& jsUjIds = jsQidUj["ujids"]; + + auto iterUjId = ujIdMap.begin(); + bool addedUjId = false; + while (iterUjId != ujIdMap.end()) { + UberJobId ujId = iterUjId->first; + auto tmStamp = iterUjId->second; + if (tmStamp > oldestTm) { + oldestTm = tmStamp; + } + + jsUjIds.push_back(ujId); + addedUjId = true; + double ageSecs = std::chrono::duration(now - tmStamp).count(); + if (ageSecs > maxLifetime) { + iterUjId = ujIdMap.erase(iterUjId); + } else { + ++iterUjId; + } + } + + if (addedUjId) { + jsDeadUj.push_back(jsQidUj); + } + + if (ujIdMap.empty() + || std::chrono::duration(now - oldestTm).count() > maxLifetime) { + iterDeadUjQid = _qIdDeadUberJobs.erase(iterDeadUjQid); + } else { + ++iterDeadUjQid; + } + } + } + + /* &&& happens in the caller now. + // Start a thread to send the message. (Maybe these should go on the qdisppool? &&&) + // put this in a different function and start the thread.&&&; + _sendStatusMsg(jsWorkerReqPtr); + */ + return jsWorkerReqPtr; +} + +WorkerQueryStatusData::Ptr WorkerQueryStatusData::createJson(nlohmann::json const& jsWorkerReq, + std::string const& replicationInstanceId, std::string const& replicationAuthKey, TIMEPOINT updateTm) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& a"); + try { + if (jsWorkerReq["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson bad version"); + return nullptr; + } + + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& b"); + auto czInfo_ = CzarContactInfo::createJson(jsWorkerReq["czar"]); + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& c"); + auto wInfo_ = WorkerContactInfo::createJson(jsWorkerReq["worker"], updateTm); + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& d"); + if (czInfo_ == nullptr || wInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson czar or worker info could not be parsed in " << jsWorkerReq); + } + auto wqsData = WorkerQueryStatusData::create(wInfo_, czInfo_, replicationInstanceId, replicationAuthKey); + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& e"); + + auto parseRes = wqsData->_parseLists(jsWorkerReq, updateTm); + if (!parseRes) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson error reading lists in " << jsWorkerReq); + return nullptr; + } + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& end"); + return wqsData; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); + } + return nullptr; +} + +bool WorkerQueryStatusData::_parseLists(nlohmann::json const& jsWorkerReq, TIMEPOINT updateTm) { + try { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& a"); + auto& jsQIdDoneKeepFiles = jsWorkerReq["qiddonekeepfiles"]; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& b"); + for (auto const& qidKeep : jsQIdDoneKeepFiles) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& b1"); + _qIdDoneKeepFiles[qidKeep] = updateTm; + } + + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& c"); + auto& jsQIdDoneDeleteFiles = jsWorkerReq["qiddonedeletefiles"]; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& d"); + for (auto const& qidDelete : jsQIdDoneDeleteFiles) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& d1"); + _qIdDoneDeleteFiles[qidDelete] = updateTm; + } + + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& e"); + auto& jsQIdDeadUberJobs = jsWorkerReq["qiddeaduberjobs"]; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f jsQIdDeadUberJobs=" << jsQIdDeadUberJobs); + // Interestingly, !jsQIdDeadUberJobs.empty() doesn't work, but .size() > 0 does. + // Not having the size() check causes issues with the for loop trying to read the + // first element of an empty list, which goes badly. + if (jsQIdDeadUberJobs.size() > 0) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f1"); + for (auto const& qDeadUjs : jsQIdDeadUberJobs) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f1a qDeadUjs=" << qDeadUjs); + QueryId qId = qDeadUjs["qid"]; + auto const& ujIds = qDeadUjs["ujids"]; + auto& mapOfUj = _qIdDeadUberJobs[qId]; + for (auto const& ujId : ujIds) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f1d1 qId=" << qId << " ujId=" << ujId); + mapOfUj[ujId] = updateTm; + } + } + } + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::_parseLists invalid ") << exc.what()); + return false; + } + return true; +} + +void WorkerQueryStatusData::addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm) { + auto& ujMap = _qIdDeadUberJobs[qId]; + for (auto const ujId : ujIds) { + ujMap[ujId] = tm; + } +} + +string WorkerQueryStatusData::dump() const { + stringstream os; + os << "ActiveWorker " << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); + return os.str(); +} + +} // namespace lsst::qserv::czar diff --git a/src/http/WorkerQueryStatusData.h b/src/http/WorkerQueryStatusData.h new file mode 100644 index 0000000000..f0f6c1aaa1 --- /dev/null +++ b/src/http/WorkerQueryStatusData.h @@ -0,0 +1,234 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_HTTP_WORKERQUERYSTATUSDATA_H +#define LSST_QSERV_HTTP_WORKERQUERYSTATUSDATA_H + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "global/clock_defs.h" +#include "global/intTypes.h" + + +// This header declarations +namespace lsst::qserv::http { + +/// &&& doc +class CzarContactInfo { +public: + using Ptr = std::shared_ptr; + std::string cName(const char* fnc) const { + return std::string("CzarContactInfo") + fnc; + } + + CzarContactInfo() = delete; + CzarContactInfo(CzarContactInfo const&) = default; + CzarContactInfo& operator=(CzarContactInfo const&) = default; + + /// &&& doc + bool compare(CzarContactInfo const& other) { + return (czName == other.czName && czId == other.czId && czPort == other.czPort && czHostName == other.czHostName); + } + + static Ptr create(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_) { + return Ptr(new CzarContactInfo(czName_, czId_, czPort_, czHostName_)); + } + + static Ptr createJson(nlohmann::json const& czarJson); + + std::string const czName; ///< czar "name" + CzarIdType const czId; ///< czar "id" + int const czPort; ///< czar "management-port" + std::string const czHostName; ///< czar "management-host-name" + + /// &&& doc + nlohmann::json serializeJson() const; + + /// &&& doc + //&&&bool parse(nlohmann::json const& czarJson); + + std::string dump() const; + /* &&& + auto& jsWCzar = jsWorkerR["czar"]; + jsWCzar["name"] = czarConfig->name(); + jsWCzar["id"]= czarConfig->id(); + jsWCzar["management-port"] = czarConfig->replicationHttpPort(); + jsWCzar["management-host-name"] = util::get_current_host_fqdn(); + */ +private: + CzarContactInfo(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_) + : czName(czName_), czId(czId_), czPort(czPort_), czHostName(czHostName_) {} +}; + + +/// &&& doc This class just contains the worker id and network communication +/// information, but it may be desirable to store connections to the +/// worker here as well. +class WorkerContactInfo { +public: + using Ptr = std::shared_ptr; + + using WCMap = std::unordered_map; + using WCMapPtr = std::shared_ptr; + + static Ptr create(std::string const& wId_, std::string const& wHost_, + std::string const& wManagementHost_, int wPort_, TIMEPOINT updateTime_) { + return Ptr(new WorkerContactInfo(wId_, wHost_, + wManagementHost_, wPort_, updateTime_)); + } + + /// &&& doc + static Ptr createJson(nlohmann::json const& workerJson, TIMEPOINT updateTime); + + /// &&& doc + nlohmann::json serializeJson() const; + + std::string cName(const char* fn) { return std::string("WorkerContactInfo::") + fn; } + + /// &&& make private + WorkerContactInfo(std::string const& wId_, std::string const& wHost_, + std::string const& wManagementHost_, int wPort_, TIMEPOINT updateTime_) + : wId(wId_), + wHost(wHost_), + wManagementHost(wManagementHost_), + wPort(wPort_) { + regUpdateTime(updateTime_); + } + std::string const wId; ///< key + std::string const wHost; ///< "host-addr" entry. + std::string const wManagementHost; ///< "management-host-name" entry. + int const wPort; ///< "management-port" entry. + + + /// Return true if all members, aside from updateTime, are equal. + bool isSameContactInfo(WorkerContactInfo const& other) const { + return (wId == other.wId && wHost == other.wHost && wManagementHost == other.wManagementHost && + wPort == other.wPort); + } + + void regUpdateTime(TIMEPOINT updateTime) { + std::lock_guard lg(_rMtx); + _regUpdate = updateTime; + } + + double timeSinceRegUpdateSeconds() const { + std::lock_guard lg(_rMtx); + double secs = std::chrono::duration(CLOCK::now() - _regUpdate).count(); + return secs; + } + + TIMEPOINT getRegUpdate() const { + std::lock_guard lg(_rMtx); + return _regUpdate; + } + + std::string dump() const; + +private: + /// Last time the registry heard from this worker. The ActiveWorker class + /// will use this to determine the worker's state. + /// &&& Store in seconds since epoch to make atomic? + TIMEPOINT _regUpdate; + + mutable std::mutex _rMtx; ///< protects _regUpdate +}; + + +/// &&& doc +class WorkerQueryStatusData { +public: + using Ptr = std::shared_ptr; + + /* &&& + enum State { + ALIVE = 0, + QUESTIONABLE, + DEAD + }; + */ + + WorkerQueryStatusData() = delete; + WorkerQueryStatusData(WorkerQueryStatusData const&) = delete; + WorkerQueryStatusData& operator=(WorkerQueryStatusData const&) = delete; + + std::string cName(const char* fName) { + return std::string("WorkerQueryStatusData::") + fName + " " + ((_wInfo == nullptr) ? "?" : _wInfo->wId); + } + + //&&&static std::string getStateStr(State st); + + static Ptr create(WorkerContactInfo::Ptr const& wInfo, CzarContactInfo::Ptr const& czInfo, + std::string const& replicationInstanceId, std::string const& replicationAuthKey) { + return Ptr(new WorkerQueryStatusData(wInfo, czInfo, replicationInstanceId, replicationAuthKey)); + } + + /// &&& doc + static Ptr createJson(nlohmann::json const& czarJson, + std::string const& replicationInstanceId, std::string const& replicationAuthKey, TIMEPOINT updateTm); + + + ~WorkerQueryStatusData() = default; + + WorkerContactInfo::Ptr getWInfo() const { return _wInfo; } + + /// &&& doc + void addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm); + + std::string dump() const; + +//&&&private: + WorkerQueryStatusData(WorkerContactInfo::Ptr const& wInfo, CzarContactInfo::Ptr const& czInfo, + std::string const& replicationInstanceId, std::string const& replicationAuthKey) + : _wInfo(wInfo), _czInfo(czInfo), + _replicationInstanceId(replicationInstanceId), _replicationAuthKey(replicationAuthKey) {} + + std::map _qIdDoneKeepFiles; ///< &&& doc - limit reached + std::map _qIdDoneDeleteFiles; ///< &&& doc -cancelled/finished + std::map> _qIdDeadUberJobs; ///< &&& doc + + /// &&& TODO:UJ Worth the effort to inform worker of killed UberJobs? + //std::map> _killedUberJobs; + + WorkerContactInfo::Ptr _wInfo; ///< &&& doc + CzarContactInfo::Ptr _czInfo; //< &&& doc + + std::string const _replicationInstanceId; ///< &&& doc + std::string const _replicationAuthKey; ///< &&& doc + + /// &&& doc + std::shared_ptr serializeJson(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime); + + /// &&& doc + /// @throws std::invalid_argument + bool _parseLists(nlohmann::json const& jsWorkerReq, TIMEPOINT updateTm); // &&& delete after basic testing +}; + +} // namespace lsst::qserv::http + +#endif // LSST_QSERV_HTTP_WORKERQUERYSTATUSDATA_H diff --git a/src/http/testStatusData.cc b/src/http/testStatusData.cc new file mode 100644 index 0000000000..97767dd9fa --- /dev/null +++ b/src/http/testStatusData.cc @@ -0,0 +1,140 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include +#include +#include + +// Qserv headers +#include "global/clock_defs.h" +#include "http/WorkerQueryStatusData.h" + +// LSST headers +#include "lsst/log/Log.h" + +// Boost unit test header +#define BOOST_TEST_MODULE RequestQuery +#include + +using namespace std; +namespace test = boost::test_tools; +using namespace lsst::qserv::http; + +BOOST_AUTO_TEST_SUITE(Suite) + +BOOST_AUTO_TEST_CASE(CzarContactInfo) { + + string const replicationInstanceId = "repliInstId"; + string const replicationAuthKey = "repliIAuthKey"; + + string const cName("czar_name"); + lsst::qserv::CzarIdType const cId = 32; + int cPort = 2022; + string const cHost("cz_host"); + + auto czarA = lsst::qserv::http::CzarContactInfo::create(cName, cId, cPort, cHost); + LOGS_ERROR("&&& a czarA=" << czarA->dump()); + + auto czarAJs = czarA->serializeJson(); + LOGS_ERROR("&&& b czarAJs=" << czarAJs); + + auto czarB = lsst::qserv::http::CzarContactInfo::createJson(czarAJs); + LOGS_ERROR("&&& c czarB=" << czarB); + BOOST_REQUIRE(czarA->compare(*czarB)); + + auto czarC = lsst::qserv::http::CzarContactInfo::create("different", cId, cPort, cHost); + BOOST_REQUIRE(!czarA->compare(*czarC)); + + auto start = lsst::qserv::CLOCK::now(); + auto workerA = WorkerContactInfo::create("sd_workerA", "host_w1", "mgmhost_a", 3421, start); + auto workerB = WorkerContactInfo::create("sd_workerB", "host_w2", "mgmhost_a", 3421, start); + auto workerC = WorkerContactInfo::create("sd_workerC", "host_w3", "mgmhost_b", 3422, start); + LOGS_ERROR("&&& d workerA=" << workerA->dump()); + + auto jsWorkerA = workerA->serializeJson(); + LOGS_ERROR("&&& e jsWorkerA=" << jsWorkerA); + auto start1Sec = start + 1s; + auto workerA1 = WorkerContactInfo::createJson(jsWorkerA, start1Sec); + LOGS_ERROR("&&& f workerA1=" << workerA1->dump()); + BOOST_REQUIRE(workerA->isSameContactInfo(*workerA1)); + + // WorkerQueryStatusData + auto wqsdA = lsst::qserv::http::WorkerQueryStatusData::create(workerA, czarA, replicationInstanceId, replicationAuthKey); + LOGS_ERROR("&&& g wqsdA=" << wqsdA->dump()); + + double timeoutAliveSecs = 100.0; + double timeoutDeadSecs = 2*timeoutAliveSecs; + double maxLifetime = 300.0; + auto jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + LOGS_ERROR("&&& h jsDataA=" << *jsDataA); + + // Check that empty lists work. + auto wqsdA1 = lsst::qserv::http::WorkerQueryStatusData::createJson(*jsDataA, replicationInstanceId, replicationAuthKey, start1Sec); + LOGS_ERROR("&&& i wqsdA1=" << wqsdA1->dump()); + auto jsDataA1 = wqsdA1->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + BOOST_REQUIRE(*jsDataA == *jsDataA1); + + + vector qIdsDelFiles = { 7, 8, 9, 15, 25, 26, 27, 30 }; + vector qIdsKeepFiles = { 1, 2, 3, 4, 6, 10, 13, 19, 33 }; + for (auto const qIdDF : qIdsDelFiles) { + wqsdA->_qIdDoneDeleteFiles[qIdDF] = start; + } + + jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + LOGS_ERROR("&&& j jsDataA=" << jsDataA); + BOOST_REQUIRE(*jsDataA != *jsDataA1); + + for (auto const qIdKF : qIdsKeepFiles) { + wqsdA->_qIdDoneKeepFiles[qIdKF] = start; + } + + wqsdA->addDeadUberJobs(12, {1, 3}, start); + + LOGS_ERROR("&&& i wqsdA=" << wqsdA->dump()); + + jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + LOGS_ERROR("&&& j jsDataA=" << *jsDataA); + + auto start5Sec = start + 5s; + auto workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createJson(*jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); + auto jsWorkerAFromJson = workerAFromJson->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); + + wqsdA->addDeadUberJobs(12, {34}, start5Sec); + wqsdA->addDeadUberJobs(91, {77}, start5Sec); + wqsdA->addDeadUberJobs(1059, {1, 4, 6, 7, 8, 10, 3, 22, 93}, start5Sec); + + jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + LOGS_ERROR("&&& k jsDataA=" << *jsDataA); + BOOST_REQUIRE(*jsDataA != *jsWorkerAFromJson); + + workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createJson(*jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); + jsWorkerAFromJson = workerAFromJson->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + LOGS_ERROR("&&& l jsWorkerAFromJson=" << *jsWorkerAFromJson); + BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); + +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/proto/CMakeLists.txt b/src/proto/CMakeLists.txt index 34fa9f77bb..e86e15216c 100644 --- a/src/proto/CMakeLists.txt +++ b/src/proto/CMakeLists.txt @@ -17,3 +17,4 @@ target_link_libraries(proto PUBLIC log protobuf ) + diff --git a/src/proto/worker.proto b/src/proto/worker.proto index 1ccefaf5f9..9bb29fb03d 100644 --- a/src/proto/worker.proto +++ b/src/proto/worker.proto @@ -66,3 +66,4 @@ message ResponseData { required uint32 rowcount = 2; required uint64 transmitsize = 3; } + diff --git a/src/protojson/ScanTableInfo.h b/src/protojson/ScanTableInfo.h index 061ea0c0f9..8cf6811fd3 100644 --- a/src/protojson/ScanTableInfo.h +++ b/src/protojson/ScanTableInfo.h @@ -36,7 +36,7 @@ namespace lsst::qserv::protojson { /// Structure to store shared scan information for a single table. /// -struct ScanTableInfo { +struct ScanTableInfo { // &&& check if still useful using ListOf = std::vector; ScanTableInfo() = default; diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 6cc8044662..15e8b26b66 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -210,7 +210,6 @@ UberJob::Ptr Executive::findUberJob(UberJobId ujId) { /// Add a new job to executive queue, if not already in. Not thread-safe. /// JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::add start"); JobQuery::Ptr jobQuery; { // Create the JobQuery and put it in the map. @@ -279,21 +278,36 @@ void Executive::addAndQueueUberJob(shared_ptr const& uj) { } else { _qdispPool->queCmd(cmd, 1); } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueJobStart end"); +} + +void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 3); + } else { + _qdispPool->queCmd(cmd, 4); + } } void Executive::runUberJob(std::shared_ptr const& uberJob) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); - bool started = uberJob->runUberJob(); - if (!started && isLimitRowComplete()) { - uberJob->callMarkCompleteFunc(false); + /// TODO:UJ delete useqdisppool, only set to false if problems during testing + bool const useqdisppool = true; + if (useqdisppool) { + auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; + + auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); + _jobStartCmdList.push_back(cmd); + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 0); + } else { + _qdispPool->queCmd(cmd, 1); + } + } else { + uberJob->runUberJob(); } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob end"); } void Executive::waitForAllJobsToStart() { LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart"); - LOGS(_log, LOG_LVL_WARN, "&&& waitForAllJobsToStart start"); // Wait for each command to start. while (true) { bool empty = _jobStartCmdList.empty(); @@ -303,7 +317,6 @@ void Executive::waitForAllJobsToStart() { cmd->waitComplete(); } LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart done"); - LOGS(_log, LOG_LVL_WARN, "&&& waitForAllJobsToStart end"); } Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { @@ -354,58 +367,60 @@ void Executive::addMultiError(int errorCode, std::string const& errorMsg, int er } } -Executive::ChunkIdJobMapType& Executive::getChunkJobMapAndInvalidate() { // &&& +Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { lock_guard lck(_chunkToJobMapMtx); - if (_chunkToJobMapInvalid.exchange(true)) { - throw util::Bug(ERR_LOC, "getChunkJobMapInvalidate called when map already invalid"); + + ChunkIdJobMapType unassignedMap; + for (auto const& [key, jobPtr] : _chunkToJobMap) { + if (!jobPtr->isInUberJob()) { + unassignedMap[key] = jobPtr; + } } - return _chunkToJobMap; + return unassignedMap; } -void Executive::addUberJobs(std::vector> const& uJobsToAdd) { // &&& - lock_guard lck(_uberJobsMtx); +void Executive::addUberJobs(std::vector> const& uJobsToAdd) { + lock_guard lck(_uberJobsMapMtx); for (auto const& uJob : uJobsToAdd) { - _uberJobs.push_back(uJob); + UberJobId ujId = uJob->getJobId(); + _uberJobsMap[ujId] = uJob; } } -bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& - - lock_guard lock(_cancelled.getMutex()); - - // If this has been cancelled, then return false. - // - if (_cancelled) return false; - - // Construct a temporary resource object to pass to ProcessRequest(). - // Affinity should be meaningless here as there should only be one instance of each worker. - XrdSsiResource::Affinity affinity = XrdSsiResource::Affinity::Default; - LOGS(_log, LOG_LVL_INFO, "&&& startUberJob uJob->workerResource=" << uJob->getWorkerResource()); - XrdSsiResource uJobResource(uJob->getWorkerResource(), "", uJob->getIdStr(), "", 0, affinity); - - // Now construct the actual query request and tie it to the jobQuery. The - // shared pointer is used by QueryRequest to keep itself alive, sloppy design. - // Note that JobQuery calls StartQuery that then calls JobQuery, yech! - // - QueryRequest::Ptr qr = QueryRequest::create(uJob); - uJob->setQueryRequest(qr); +string Executive::dumpUberJobCounts() const { + stringstream os; + os << "exec=" << getIdStr(); + int totalJobs = 0; + { + lock_guard ujmLck(_uberJobsMapMtx); + for (auto const& [ujKey, ujPtr] : _uberJobsMap) { + int jobCount = ujPtr->getJobCount(); + totalJobs += jobCount; + os << "{" << ujKey << ":" << ujPtr->getIdStr() << " jobCount=" << jobCount << "}"; + } + } + { + lock_guard jmLck(_jobMapMtx); + os << " ujTotalJobs=" << totalJobs << " execJobs=" << _jobMap.size(); + } + return os.str(); +} - // Start the query. The rest is magically done in the background. - // - getXrdSsiService()->ProcessRequest(*(qr.get()), uJobResource); - return true; +void Executive::assignJobsToUberJobs() { + auto uqs = _userQuerySelect.lock(); + if (uqs != nullptr) { + uqs->buildAndSendUberJobs(); + } } -JobQuery::Ptr Executive::getSharedPtrForRawJobPtr(JobQuery* jqRaw) { //&&& - assert(jqRaw != nullptr); - int jobId = jqRaw->getIdInt(); - lock_guard lockJobMap(_jobMapMtx); - auto iter = _jobMap.find(jobId); - if (iter == _jobMap.end()) { - throw util::Bug(ERR_LOC, "Could not find the entry for jobId=" + to_string(jobId)); +void Executive::addMultiError(int errorCode, std::string const& errorMsg, int errorState) { + util::Error err(errorCode, errorMsg, errorState); + { + lock_guard lock(_errorsMutex); + _multiError.push_back(err); + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) + " multiError:" << _multiError.size() << ":" << _multiError); } - JobQuery::Ptr jq = iter->second; - return jq; } /// Add a JobQuery to this Executive. @@ -620,6 +635,16 @@ void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { } } +void Executive::sendWorkerCancelMsg(bool deleteResults) { + // TODO:UJ need to send a message to the worker that the query is cancelled and all result files + // should be delete + LOGS(_log, LOG_LVL_ERROR, + "TODO:UJ NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId " + "+ " + "queryId. " + << deleteResults); +} + int Executive::getNumInflight() const { unique_lock lock(_incompleteJobsMutex); return _incompleteJobs.size(); diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 63506f35f8..bb50645d81 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -50,10 +50,6 @@ #include "util/threadSafe.h" #include "util/ThreadPool.h" - -// Forward declarations -class XrdSsiService; - namespace lsst::qserv::qmeta { class QProgress; class QProgressHistory; @@ -89,6 +85,7 @@ class QdispPool; // This header declarations namespace lsst::qserv::qdisp { + /// class Executive manages the execution of jobs for a UserQuery. class Executive : public std::enable_shared_from_this { public: @@ -121,6 +118,18 @@ class Executive : public std::enable_shared_from_this { /// Find the UberJob with `ujId`. std::shared_ptr findUberJob(UberJobId ujId); + std::string cName(const char* funcName = "") { return std::string("Executive::") + funcName; } + + /// Set the UserQuerySelect object for this query so this Executive can ask it to make new + /// UberJobs in the future, if needed. + void setUserQuerySelect(std::shared_ptr const& uqs) { _userQuerySelect = uqs; } + + /// Return a map that only contains Jobs not assigned to an UberJob. + ChunkIdJobMapType unassignedChunksInQuery(); + + /// Find the UberJob with `ujId`. + std::shared_ptr findUberJob(UberJobId ujId); + /// Add an item with a reference number std::shared_ptr add(JobDescription::Ptr const& s); @@ -130,6 +139,9 @@ class Executive : public std::enable_shared_from_this { /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. void queueFileCollect(std::shared_ptr const& cmd); + /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. + void queueFileCollect(PriorityCommand::Ptr const& cmd); + /// Waits for all jobs on _jobStartCmdList to start. This should not be called /// before ALL jobs have been added to the pool. void waitForAllJobsToStart(); @@ -204,6 +216,9 @@ class Executive : public std::enable_shared_from_this { /// Add an error code and message that may be displayed to the user. void addMultiError(int errorCode, std::string const& errorMsg, int errState); + /// Add an error code and message that may be displayed to the user. + void addMultiError(int errorCode, std::string const& errorMsg, int errState); + std::string dumpUberJobCounts() const; // The below value should probably be based on the user query, with longer sleeps for slower queries. @@ -329,14 +344,17 @@ class Executive : public std::enable_shared_from_this { std::map> _uberJobsMap; mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. - // Add a job to the _chunkToJobMap //&&&uj - void _addToChunkJobMap(std::shared_ptr const& job); //&&&uj - /// _chunkToJobMap is created once and then destroyed when used. - std::atomic _chunkToJobMapInvalid{false}; ///< true indicates the map is no longer valid. //&&&uj - std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap //&&&uj - ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId //&&&uj - std::vector> _uberJobs; ///< List of UberJobs //&&&uj - std::mutex _uberJobsMtx; ///< protects _uberJobs. //&&&uj + // Add a job to the _chunkToJobMap + // TODO:UJ This may need review as large changes were made to this part of the code. + // code is no longer destructive to _chunkToJobMap + void _addToChunkJobMap(std::shared_ptr const& job); + std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap + ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId + + /// Map of all UberJobs. Failed UberJobs remain in the map as new ones are created + /// to handle failed UberJobs. + std::map> _uberJobsMap; + mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. diff --git a/src/qdisp/JobBase.cc b/src/qdisp/JobBase.cc new file mode 100644 index 0000000000..a5ef5a8c8f --- /dev/null +++ b/src/qdisp/JobBase.cc @@ -0,0 +1,54 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "qdisp/JobBase.h" + +// System headers +#include + +// Qserv headers + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobBase"); +} + +namespace lsst { namespace qserv { namespace qdisp { + +std::ostream& JobBase::dumpOS(std::ostream& os) const { + os << "JobBase no data members"; + return os; +} + +std::string JobBase::dump() const { + std::ostringstream os; + dumpOS(os); + return os.str(); +} + +std::ostream& operator<<(std::ostream& os, JobBase const& jb) { return jb.dumpOS(os); } + +}}} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h new file mode 100644 index 0000000000..1a42394578 --- /dev/null +++ b/src/qdisp/JobBase.h @@ -0,0 +1,78 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_QDISP_JOBBASE_H +#define LSST_QSERV_QDISP_JOBBASE_H + +// System headers +#include +#include + +// Qserv headers +#include "global/intTypes.h" + +namespace lsst::qserv::qmeta { +class JobStatus; +} + +// This header declarations +namespace lsst::qserv::qdisp { + +class Executive; +class QdispPool; +class ResponseHandler; +class QueryRequest; + +/// Base class for JobQuery and UberJob. +/// TODO:UJ This could use a lot of cleanup. Once UberJobs are fully in effect, there's no need +/// for this base class as it won't be possible to send a JobQuery to a worker without +/// putting it in an UberJob first. The UberJob is a wrapper that stores worker contact +/// info. +// &&& delete this class as JobQuery and UberJob should no longer have much in common +class JobBase : public std::enable_shared_from_this { +public: + using Ptr = std::shared_ptr; + + JobBase() = default; + JobBase(JobBase const&) = delete; + JobBase& operator=(JobBase const&) = delete; + virtual ~JobBase() = default; + + virtual QueryId getQueryId() const = 0; + virtual UberJobId getJobId() const = 0; + virtual std::string const& getIdStr() const = 0; + virtual std::shared_ptr getQdispPool() = 0; + //&&& virtual std::string const& getPayload() const = 0; ///< const& in return type is essential for xrootd + virtual std::shared_ptr getRespHandler() = 0; + virtual std::shared_ptr getStatus() = 0; + virtual bool getScanInteractive() const = 0; + virtual bool isQueryCancelled() = 0; + virtual void callMarkCompleteFunc(bool success) = 0; + virtual std::shared_ptr getExecutive() = 0; + + virtual std::ostream& dumpOS(std::ostream& os) const; + + std::string dump() const; + friend std::ostream& operator<<(std::ostream& os, JobBase const& jb); +}; + +} // namespace lsst::qserv::qdisp + +#endif // LSST_QSERV_QDISP_JOBBASE_H diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 1d205434ec..a5bd734157 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -89,7 +89,6 @@ bool JobDescription::incrAttemptCount(std::shared_ptr const& exec, bo << " max=" << MAX_JOB_ATTEMPTS); return false; } - return true; } diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index acee88cdfb..c2d61759d4 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -90,7 +90,9 @@ class JobDescription { void resetJsForWorker() { _jsForWorker.reset(); } - bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj + std::shared_ptr getJsForWorker() { return _jsForWorker; } + + void resetJsForWorker() { _jsForWorker.reset(); } // TODO:UJ may need mutex for _jsForWorker //&&& friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 3fd1ce1bbf..0ddc24549f 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -43,28 +43,16 @@ using namespace std; namespace lsst::qserv::qdisp { -/* &&& JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - qmeta::JobStatus::Ptr const& jobStatus, QueryId qid) - : _executive(executive), - _jobDescription(jobDescription), - _jobStatus(jobStatus), - _qid(qid), - _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { - LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); -} -*/ - -JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - JobStatus::Ptr const& jobStatus, shared_ptr const& markCompleteFunc, - QueryId qid) + qmeta::JobStatus::Ptr const& jobStatus, + shared_ptr const& markCompleteFunc, QueryId qid) : JobBase(), _executive(executive), _jobDescription(jobDescription), _markCompleteFunc(markCompleteFunc), _jobStatus(jobStatus), _qid(qid), - _idStr(QueryIdHelper::makeIdStr(qid, getIdInt())) { + _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { _qdispPool = executive->getQdispPool(); LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } @@ -72,7 +60,7 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_TRACE, "~JobQuery QID=" << _idStr); } /// Cancel response handling. Return true if this is the first time cancel has been called. -bool JobQuery::cancel(bool superfluous) { +bool JobQuery::cancel(bool superfluous) { /// &&& This can probably be simplified more QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); if (_cancelled.exchange(true) == false) { LOGS(_log, LOG_LVL_TRACE, "JobQuery::cancel() " << superfluous); diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 4f37dcac89..f069558673 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -61,9 +61,9 @@ class JobQuery { QueryId getQueryId() const { return _qid; } JobId getJobId() const { return _jobDescription->id(); } std::string const& getIdStr() const { return _idStr; } + JobDescription::Ptr getDescription() { return _jobDescription; } qmeta::JobStatus::Ptr getStatus() { return _jobStatus; } - bool cancel(bool superfluous = false); bool isQueryCancelled() override; @@ -121,6 +121,17 @@ class JobQuery { return _uberJobId >= 0; } + /// @return true if _uberJobId was set, it can only be set if it is unassigned + /// or by the current owner. + /// NOTE: _rmutex must be held before calling this + bool _setUberJobId(UberJobId ujId); + + /// NOTE: _rmutex must be held before calling this + UberJobId _getUberJobId() const { return _uberJobId; } + + /// NOTE: _rmutex must be held before calling this + bool _isInUberJob() const { return _uberJobId >= 0; } + // Values that don't change once set. std::weak_ptr _executive; /// The job description needs to survive until the task is complete. diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index dab0942b98..7c21b6175a 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -77,6 +77,17 @@ class ResponseHandler { /// Add the error to the error output if it is the first error. virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; + /// Collect result data from the worker and merge it with the query result table. + /// @return success - true if the operation was successful + /// @return shouldCancel - if success was false, this being true indicates there + /// was an unrecoverable error in table writing and the query + /// should be cancelled. + virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) = 0; + + /// Add the error to the error output if it is the first error. + virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; + /// Signal an unrecoverable error condition. No further calls are expected. virtual void errorFlush(std::string const& msg, int code) = 0; diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index de713dac12..61d7c31f03 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -205,6 +205,28 @@ void UberJob::_unassignJobs() { _jobs.clear(); } +void UberJob::_unassignJobs() { + lock_guard lck(_jobsMtx); + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " exec is null"); + return; + } + for (auto&& job : _jobs) { + string jid = job->getIdStr(); + if (!job->unassignFromUberJob(getJobId())) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not unassign job=" << jid << " cancelling"); + exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "unable to re-assign " + jid, + util::ErrorCode::INTERNAL); + exec->squash(); + return; + } + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); + } + _jobs.clear(); +} + bool UberJob::isQueryCancelled() { auto exec = _executive.lock(); if (exec == nullptr) { @@ -244,6 +266,24 @@ bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg void UberJob::callMarkCompleteFunc(bool success) { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " success=" << success); + // Overwriting errors is probably not a good idea. + if (currentState >= qmeta::JobStatus::CANCEL && currentState < qmeta::JobStatus::COMPLETE) { + LOGS(_log, LOG_LVL_WARN, + getIdStr() << "UberJob::" << __func__ << " already error current=" + << _jobStatus->stateStr(currentState) << " new=" << _jobStatus->stateStr(newState)); + return false; + } + + _jobStatus->updateInfo(getIdStr(), newState, msg); + for (auto&& jq : _jobs) { + jq->getStatus()->updateInfo(jq->getIdStr(), newState, msg); + } + return true; +} + +void UberJob::callMarkCompleteFunc(bool success) { + LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); + lock_guard lck(_jobsMtx); // Need to set this uberJob's status, however exec->markCompleted will set // the status for each job when it is called. @@ -495,6 +535,197 @@ void UberJob::killUberJob() { return; } +/// Retrieve and process a result file using the file-based protocol +/// Uses a copy of JobQuery::Ptr instead of _jobQuery as a call to cancel() would reset _jobQuery. +json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); + + if (isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); + return _importResultError(true, "cancelled", "Query cancelled"); + } + + auto exec = _executive.lock(); + if (exec == nullptr || exec->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive or cancelled"); + return _importResultError(true, "cancelled", "Query cancelled - no executive"); + } + + if (exec->isLimitRowComplete()) { + int dataIgnored = exec->incrDataIgnoredCount(); + if ((dataIgnored - 1) % 1000 == 0) { + LOGS(_log, LOG_LVL_INFO, + "UberJob ignoring, enough rows already " << "dataIgnored=" << dataIgnored); + } + return _importResultError(false, "rowLimited", "Enough rows already"); + } + + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileSize=" << fileSize); + + bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); + if (!statusSet) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " setStatusFail could not set status to RESPONSE_READY"); + return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); + } + + JobBase::Ptr jBaseThis = shared_from_this(); + weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); + + // TODO:UJ lambda may not be the best way to do this, alsocheck synchronization - may need a mutex for + // merging. + auto fileCollectFunc = [ujThis, fileUrl, rowCount](util::CmdData*) { + auto ujPtr = ujThis.lock(); + if (ujPtr == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, + "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + return; + } + uint64_t resultRows = 0; + auto [flushSuccess, flushShouldCancel] = + ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); + LOGS(_log, LOG_LVL_DEBUG, ujPtr->cName(__func__) << "::fileCollectFunc"); + if (!flushSuccess) { + // This would probably indicate malformed file+rowCount or + // writing the result table failed. + ujPtr->_importResultError(flushShouldCancel, "mergeError", "merging failed"); + } + + // At this point all data for this job have been read, there's no point in + // having XrdSsi wait for anything. + ujPtr->_importResultFinish(resultRows); + }; + + auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); + exec->queueFileCollect(cmd); + + // If the query meets the limit row complete complete criteria, it will start + // squashing superfluous results so the answer can be returned quickly. + + json jsRet = {{"success", 1}, {"errortype", ""}, {"note", "queued for collection"}}; + return jsRet; +} + +json UberJob::workerError(int errorCode, string const& errorMsg) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " errcode=" << errorCode << " errmsg=" << errorMsg); + + bool const deleteData = true; + bool const keepData = !deleteData; + auto exec = _executive.lock(); + if (exec == nullptr || isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); + return _workerErrorFinish(deleteData, "cancelled"); + } + + if (exec->isLimitRowComplete()) { + int dataIgnored = exec->incrDataIgnoredCount(); + if ((dataIgnored - 1) % 1000 == 0) { + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " ignoring, enough rows already " + << "dataIgnored=" << dataIgnored); + } + return _workerErrorFinish(keepData, "none", "limitRowComplete"); + } + + // Currently there are no detectable recoverable errors from workers. The only + // error that a worker could send back that may possibly be recoverable would + // be a missing table error, which is not trivial to detect. A worker local + // database error may also qualify. + // TODO:UJ see if recoverable errors can be detected on the workers, or + // maybe allow a single retry before sending the error back to the user? + bool recoverableError = false; + recoverableError = true; // TODO:UJ delete after testing + if (recoverableError) { + // The czar should have new maps before the the new UberJob(s) for + // these Jobs are created. (see Czar::_monitor) + _unassignJobs(); + } else { + // Get the error message to the user and kill the user query. + int errState = util::ErrorCode::MYSQLEXEC; + getRespHandler()->flushHttpError(errorCode, errorMsg, errState); + exec->addMultiError(errorCode, errorMsg, errState); + exec->squash(); + } + + string errType = to_string(errorCode) + ":" + errorMsg; + return _workerErrorFinish(deleteData, errType, ""); +} + +json UberJob::_importResultError(bool shouldCancel, string const& errorType, string const& note) { + json jsRet = {{"success", 0}, {"errortype", errorType}, {"note", note}}; + // In all cases, the worker should delete the file as this czar will not ask for it. + + auto exec = _executive.lock(); + if (exec != nullptr) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " shouldCancel=" << shouldCancel << " errorType=" << errorType << " " + << note); + if (shouldCancel) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failing jobs"); + callMarkCompleteFunc(false); // all jobs failed, no retry + exec->squash(); + } else { + /// - each JobQuery in _jobs needs to be flagged as needing to be + /// put in an UberJob and it's attempt count increased and checked + /// against the attempt limit. + /// - executive needs to be told to make new UberJobs until all + /// JobQueries are being handled by an UberJob. + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " reassigning jobs"); + _unassignJobs(); + exec->assignJobsToUberJobs(); + } + } else { + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " already cancelled shouldCancel=" << shouldCancel + << " errorType=" << errorType << " " << note); + } + return jsRet; +} + +nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + /// If this is called, the file has been collected and the worker should delete it + /// + /// This function should call markComplete for all jobs in the uberjob + /// and return a "success:1" json message to be sent to the worker. + bool const statusSet = + setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); + if (!statusSet) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " failed to set status " << getIdStr()); + return {{"success", 0}, {"errortype", "statusMismatch"}, {"note", "failed to set status"}}; + } + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); + return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + } + + bool const success = true; + callMarkCompleteFunc(success); // sets status to COMPLETE + exec->addResultRows(resultRows); + exec->checkLimitRowComplete(); + + json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; + return jsRet; +} + +nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, + std::string const& note) { + // If this is called, the file has been collected and the worker should delete it + // + // Should this call markComplete for all jobs in the uberjob??? + // TODO:UJ Only recoverable errors would be: communication failure, or missing table ??? + // Return a "success:1" json message to be sent to the worker. + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); + return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + } + + json jsRet = {{"success", 1}, {"deletedata", deleteData}, {"errortype", ""}, {"note", ""}}; + return jsRet; +} + std::ostream& UberJob::dumpOS(std::ostream& os) const { os << "(jobs sz=" << _jobs.size() << "("; lock_guard lockJobsMtx(_jobsMtx); diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 6333b3eb23..e1ef5f769b 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -156,7 +156,6 @@ class UberJob : public std::enable_shared_from_this { // changed to better represent UberJobs std::string _payload; ///< XrdSsi message to be sent to the _workerResource. TODO:UJ remove when possible - std::weak_ptr _executive; std::shared_ptr _respHandler; QueryId const _queryId; diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index 2e13f8b458..969971f5ed 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -23,14 +23,6 @@ target_link_libraries(qmeta PUBLIC mysqlclient_r http ) -# &&& see if there's a way to remove some target_link_libraries -# &&& ccontrol -# &&& czar -# &&& parser -# &&& rproc -# &&& qana -# &&& qproc -# &&& query install(TARGETS qmeta) diff --git a/src/qmeta/QMeta.h b/src/qmeta/QMeta.h index 62f87ca4a9..8bd55c7a41 100644 --- a/src/qmeta/QMeta.h +++ b/src/qmeta/QMeta.h @@ -101,50 +101,6 @@ class QMeta { */ typedef std::vector > TableNames; -#if 0 //&&& - /** - * The structure ChunkMap encapsulates a disposition of chunks at Qserv workers - * along with a time when the map was updated. - * - * Here is an example on how to using the map for getting info on all chunks in - * the given context: - * @code - * std::string const worker = "worker-001"; - * std::string const database = "LSST-DR01"; - * std::string const table = "Object"; - * - * ChunkMap const& chunkMap = ...; - * for (auto const& [chunk, size] : chunkMap[worker][database][table]) { - * ... - * } - * @endcode - */ - struct QMetaChunkMap { - /// @return 'true' if the map is empty (or constructed using the default constructor) - bool empty() const { - return workers.empty() || (std::chrono::time_point() == updateTime); - } - - // NOTE: Separate types were added here for the sake of clarity to avoid - // a definition of the unreadable nested map. - - struct ChunkInfo { - unsigned int chunk = 0; ///< The chunk number - size_t size = 0; ///< The file size (in bytes) of the chunk table - }; - typedef std::vector Chunks; ///< Collection of chunks - typedef std::map Tables; ///< tables-to-chunks - typedef std::map Databases; ///< Databases-to-tables - typedef std::map Workers; ///< Workers-to-databases - - /// The chunk disposition map for all workers. - Workers workers; - - /// The last time the map was updated (since UNIX Epoch). - TIMEPOINT updateTime; - }; -#endif //&&& - /** * Create QMeta instance from configuration dictionary. * diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h new file mode 100644 index 0000000000..1e3bfd3bef --- /dev/null +++ b/src/qproc/TaskMsgFactory.h @@ -0,0 +1,75 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2013-2017 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +#ifndef LSST_QSERV_QPROC_TASKMSGFACTORY_H +#define LSST_QSERV_QPROC_TASKMSGFACTORY_H +/** + * @file + * + * @brief TaskMsgFactory is a factory for TaskMsg (protobuf) objects. + * + * @author Daniel L. Wang, SLAC + */ + +// System headers +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// Qserv headers +#include "global/DbTable.h" +#include "global/intTypes.h" +#include "proto/worker.pb.h" +#include "qmeta/types.h" + +namespace lsst::qserv::qproc { + +class ChunkQuerySpec; + +/// TaskMsgFactory is a factory for TaskMsg (protobuf) objects. +/// All member variables must be thread safe. +/// &&& fix doc +class TaskMsgFactory { +public: + using Ptr = std::shared_ptr; + + TaskMsgFactory() = default; + virtual ~TaskMsgFactory() {} + + /// Make and return the json message for a single Job. + virtual std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, + std::string const& chunkResultName, QueryId queryId, + int jobId, int attemptCount, qmeta::CzarId czarId); + +private: + /// Make a json message for a single fragment. + void _addFragmentJson(nlohmann::json& jsFragments, std::string const& resultName, + DbTableSet const& subChunkTables, std::vector const& subChunkIds, + std::vector const& queries); +}; + +} // namespace lsst::qserv::qproc + +#endif // LSST_QSERV_QPROC_TASKMSGFACTORY_H diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index bb21066f70..a0c146e65a 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -277,6 +277,117 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response return ret; } +bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { + UberJobId const uJobId = uberJob->getJobId(); + std::string queryIdJobStr = uberJob->getIdStr(); + if (!_queryIdStrSet) { + _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); + } + + // Nothing to do if size is zero. + if (responseData.row_size() == 0) { + return true; + } + + // Do nothing if the query got cancelled for any reason. + if (uberJob->isQueryCancelled()) { + return true; + } + auto executive = uberJob->getExecutive(); + if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { + return true; + } + + std::unique_ptr semaLock; + if (_dbEngine != MYISAM) { + // needed for parallel merging with INNODB and MEMORY + semaLock.reset(new util::SemaLock(*_semaMgrConn)); + } + + TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, + bool success) { + if (!success) return; + if (std::chrono::duration const seconds = end - start; seconds.count() > 0) { + qdisp::CzarStats::get()->addXRootDSSIRecvRate(bytes / seconds.count()); + } + }; + auto tct = make_shared>(cbf); + + bool ret = false; + // Add columns to rows in virtFile. + util::Timer virtFileT; + virtFileT.start(); + // UberJobs only get one attempt + int resultJobId = makeJobIdAttempt(uberJob->getJobId(), 0); + ProtoRowBuffer::Ptr pRowBuffer = std::make_shared( + responseData, resultJobId, _jobIdColName, _jobIdSqlType, _jobIdMysqlType); + std::string const virtFile = _infileMgr.prepareSrc(pRowBuffer); + std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); + virtFileT.stop(); + + // If the job attempt is invalid, exit without adding rows. + // It will wait here if rows need to be deleted. + if (_invalidJobAttemptMgr.incrConcurrentMergeCount(resultJobId)) { + return true; + } + + size_t const resultSize = responseData.transmitsize(); + size_t tResultSize; + { + std::lock_guard resultSzLock(_mtxResultSizeMtx); + _perJobResultSize[uJobId] += resultSize; + tResultSize = _totalResultSize + _perJobResultSize[uJobId]; + } + if (tResultSize > _maxResultTableSizeBytes) { + std::ostringstream os; + os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable + << " is too large at " << tResultSize << " bytes, max allowed size is " << _maxResultTableSizeBytes + << " bytes"; + LOGS(_log, LOG_LVL_ERROR, os.str()); + _error = util::Error(-1, os.str(), -1); + return false; + } + + tct->addToValue(resultSize); + tct->setSuccess(); + tct.reset(); // stop transmit recieve timer before merging happens. + + qdisp::CzarStats::get()->addTotalBytesRecv(resultSize); + qdisp::CzarStats::get()->addTotalRowsRecv(responseData.rowcount()); + + // Stop here (if requested) after collecting stats on the amount of data collected + // from workers. + if (_config.debugNoMerge) { + return true; + } + + auto start = std::chrono::system_clock::now(); + switch (_dbEngine) { + case MYISAM: + ret = _applyMysqlMyIsam(infileStatement, resultSize); + break; + case INNODB: // Fallthrough + case MEMORY: + ret = _applyMysqlInnoDb(infileStatement, resultSize); + break; + default: + throw std::invalid_argument("InfileMerger::_dbEngine is unknown =" + engineToStr(_dbEngine)); + } + auto end = std::chrono::system_clock::now(); + auto mergeDur = std::chrono::duration_cast(end - start); + LOGS(_log, LOG_LVL_DEBUG, + "mergeDur=" << mergeDur.count() << " sema(total=" << _semaMgrConn->getTotalCount() + << " used=" << _semaMgrConn->getUsedCount() << ")"); + if (not ret) { + LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); + } + _invalidJobAttemptMgr.decrConcurrentMergeCount(); + + LOGS(_log, LOG_LVL_DEBUG, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); + + return ret; +} + bool InfileMerger::_applyMysqlMyIsam(std::string const& query, size_t resultSize) { std::unique_lock lock(_mysqlMutex); for (int j = 0; !_mysqlConn.connected(); ++j) { diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index 77eb9f796d..9cc8704b02 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -108,6 +108,9 @@ class InfileMerger { /// Merge the result data collected over Http. bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + /// Merge the result data collected over Http. + bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + /// Indicate the merge for the job is complete. void mergeCompleteFor(int jobId); diff --git a/src/wbase/MsgProcessor.h b/src/wbase/MsgProcessor.h new file mode 100644 index 0000000000..8b48de7ec4 --- /dev/null +++ b/src/wbase/MsgProcessor.h @@ -0,0 +1,64 @@ + +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2011-2016 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +/// MsgProcessor.h +#ifndef LSST_QSERV_WBASE_MSG_PROCESSOR_H +#define LSST_QSERV_WBASE_MSG_PROCESSOR_H + +// System headers +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// Forward declarations +namespace lsst::qserv::wbase { +class Task; +struct TaskSelector; +class WorkerCommand; +} // namespace lsst::qserv::wbase + +namespace lsst::qserv::wbase { + +/// MsgProcessor implementations handle incoming Task objects. +struct MsgProcessor { // &&& delete file if possible + virtual ~MsgProcessor() {} + + /// Process a group of query processing tasks. + virtual void processTasks(std::vector> const& tasks) = 0; // &&& delete + + /// Process a managememt command + virtual void processCommand(std::shared_ptr const& command) = 0; // &&& can this be deleted + + /** + * Retreive the status of queries being processed by the worker. + * @param taskSelector Task selection criterias. + * @return a JSON representation of the object's status for the monitoring + */ + virtual nlohmann::json statusToJson(wbase::TaskSelector const& taskSelector) = 0; // &&& can this be deleted +}; + +} // namespace lsst::qserv::wbase + +#endif // LSST_QSERV_WBASE_MSG_PROCESSOR_H diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index ceb6f06296..7a4ca7233f 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -70,7 +70,6 @@ using namespace nlohmann; namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.Task"); - size_t const MB_SIZE_BYTES = 1024 * 1024; } // namespace @@ -171,6 +170,7 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " created"); } + Task::~Task() {} std::vector Task::createTasksFromUberJobMsg( @@ -271,6 +271,110 @@ std::vector Task::createTasksFromUberJobMsg( return vect; } +std::vector Task::createTasksForChunk( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, proto::ScanInfo const& scanInfo, + bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, + std::shared_ptr const& sqlConnMgr, + std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { + QueryId qId = ujData->getQueryId(); + UberJobId ujId = ujData->getUberJobId(); + + UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + + string funcN(__func__); + funcN += " QID=" + to_string(qId) + " "; + + vector vect; + for (auto const& job : jsJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBodyJSON rbJobDesc(jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + auto const jdCzarId = rbJobDesc.required("czarId"); + auto const jdQueryId = rbJobDesc.required("queryId"); + if (jdQueryId != qId) { + throw TaskException(ERR_LOC, string("ujId=") + to_string(ujId) + " qId=" + to_string(qId) + + " QueryId mismatch Job qId=" + to_string(jdQueryId)); + } + auto const jdJobId = rbJobDesc.required("jobId"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_TRACE, + funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); + + auto const jdQueryFragments = rbJobDesc.required("queryFragments"); + int fragmentNumber = 0; + for (auto const& frag : jdQueryFragments) { + vector fragSubQueries; + vector fragSubchunkIds; + vector fragSubTables; + LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); + http::RequestBodyJSON rbFrag(frag); + auto const& jsQueries = rbFrag.required("queries"); + // TODO:UJ move to uberjob???, these should be the same for all jobs + for (auto const& subQ : jsQueries) { + http::RequestBodyJSON rbSubQ(subQ); + auto const subQuery = rbSubQ.required("subQuery"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); + fragSubQueries.push_back(subQuery); + } + auto const& resultTable = rbFrag.required("resultTable"); + auto const& jsSubIds = rbFrag.required("subchunkIds"); + for (auto const& scId : jsSubIds) { + fragSubchunkIds.push_back(scId); + } + auto const& jsSubTables = rbFrag.required("subchunkTables"); + + for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? + http::RequestBodyJSON rbScDbTable(scDbTable); + string scDb = rbScDbTable.required("scDb"); + string scTable = rbScDbTable.required("scTable"); + TaskDbTbl scDbTbl(scDb, scTable); + fragSubTables.push_back(scDbTbl); + } + + for (string const& fragSubQ : fragSubQueries) { + size_t templateId = userQueryInfo->addTemplate(fragSubQ); + if (fragSubchunkIds.empty()) { + bool const noSubchunks = false; + int const subchunkId = -1; + auto task = Task::Ptr(new Task( + ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, userQueryInfo, + templateId, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, + maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); + vect.push_back(task); + } else { + for (auto subchunkId : fragSubchunkIds) { + bool const hasSubchunks = true; + auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, + fragmentNumber, userQueryInfo, templateId, + hasSubchunks, subchunkId, jdQuerySpecDb, scanInfo, + scanInteractive, maxTableSizeMb, fragSubTables, + fragSubchunkIds, sendChannel, resultsHttpPort)); + vect.push_back(task); + } + } + } + ++fragmentNumber; + } + } + + for (auto taskPtr : vect) { + // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. + taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, + sqlConnMgr, queriesAndChunks)); + } + return vect; +} + std::vector Task::createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, std::shared_ptr const& sendChannel, int maxTableSizeMb, diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 651046474c..3cf22aacad 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -22,11 +22,10 @@ // Class header #include "wbase/UberJobData.h" -#include "../wcontrol/WCzarInfoMap.h" +#include "wcontrol/WCzarInfoMap.h" // System headers // Third party headers -#include "boost/filesystem.hpp" // LSST headers #include "lsst/log/Log.h" @@ -109,6 +108,14 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount cName(__func__) << " _foreman was null, which should only happen in unit tests"); } + string workerIdStr; + if (_foreman != nullptr) { + workerIdStr = _foreman->chunkInventory()->id(); + } else { + workerIdStr = "dummyWorkerIdStr"; + LOGS(_log, LOG_LVL_INFO, funcN << " _foreman was null, which should only happen in unit tests"); + } + json request = {{"version", http::MetaModule::version}, {"workerid", workerIdStr}, {"auth_key", _authKey}, diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index 703c039b24..f5626acd67 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -124,6 +124,21 @@ class UberJobData : public std::enable_shared_from_this { std::string resultFilePath() const; std::string resultFileHttpUrl() const; + /// Add the tasks defined in the UberJob to this UberJobData object. + void addTasks(std::vector> const& tasks) { + _ujTasks.insert(_ujTasks.end(), tasks.begin(), tasks.end()); + } + + /// Let the czar know the result is ready. + void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, + uint64_t headerCount); // TODO:UJ remove headerCount + + /// Let the Czar know there's been a problem. + bool responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); + + std::string getIdStr() const { return _idStr; } + std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } + private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, int czarPort, uint64_t queryId, int rowLimit, uint64_t maxTableSizeBytes, diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index a2e381ffae..e0e49edc2c 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -238,6 +238,7 @@ bool QueryRunner::_dispatchChannel() { LOGS(_log, LOG_LVL_TRACE, "QR " << taskSched->histTimeOfRunningTasks->getString("run")); } else { LOGS(_log, LOG_LVL_ERROR, "QR runtaskSched == nullptr"); + LOGS(_log, LOG_LVL_ERROR, "&&&DASH QR runtaskSched == nullptr"); } double runTimeSeconds = primeT.getElapsed(); double subchunkRunTimeSeconds = subChunkT.getElapsed(); diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 1bc2f885c6..7447cff8bc 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -54,6 +54,7 @@ namespace util = lsst::qserv::util; using lsst::qserv::mysql::MySqlConfig; using lsst::qserv::mysql::MySqlConnection; + using lsst::qserv::wbase::FileChannelShared; using lsst::qserv::wbase::SendChannel; using lsst::qserv::wbase::Task; @@ -128,6 +129,106 @@ struct Fixture { return jsJobMsgPtr; } + struct MsgInfo { + string const db = "LSST"; + string const table = "Object"; + string const qry = "SELECT AVG(yFlux_PS) from LSST.Object_3240"; + int const chunkId = 3240; + int const czarId = 5; + string const czarName = "cz5"; + string const czarHostName = "cz5host"; + int const czarPort = 3437; + string const targWorkerId = "a_worker"; + // &&& make mock foreman instead of nullptr? + std::shared_ptr foreman; + int const queryId = 23; + int const jobId = 1; + int const uberJobId = 1; + int const attemptCount = 1; + int const scanRating = 1; + bool const scanInteractive = false; + int const maxTableSize = 5000; + bool const lockInMemory = false; + string const resultName = "resName"; + string const authKey = "noAuthKey"; + }; + + shared_ptr newTaskJson(MsgInfo const& mInfo) { + // Derived from TaskMsgFactory::makeMsgJson + + auto jsJobMsgPtr = std::shared_ptr( + new nlohmann::json({{"czarId", mInfo.czarId}, + {"queryId", mInfo.queryId}, + {"jobId", mInfo.jobId}, + {"attemptCount", mInfo.attemptCount}, + {"querySpecDb", mInfo.db}, + {"scanPriority", mInfo.scanRating}, + {"scanInteractive", mInfo.scanInteractive}, + {"maxTableSize", mInfo.maxTableSize}, + {"chunkScanTables", nlohmann::json::array()}, + {"chunkId", mInfo.chunkId}, + {"queryFragments", nlohmann::json::array()}})); + + auto& jsJobMsg = *jsJobMsgPtr; + + auto& chunkScanTables = jsJobMsg["chunkScanTables"]; + /* &&& + for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { + nlohmann::json cst = {{"db", sTbl.db}, + {"table", sTbl.table}, + {"lockInMemory", sTbl.lockInMemory}, + {"tblScanRating", sTbl.scanRating}}; + chunkScanTables.push_back(move(cst)); + } + */ + nlohmann::json cst = {{"db", mInfo.db}, + {"table", mInfo.table}, + {"lockInMemory", mInfo.lockInMemory}, + {"tblScanRating", mInfo.scanRating}}; + chunkScanTables.push_back(move(cst)); + + + auto& jsFragments = jsJobMsg["queryFragments"]; + /* &&& + if (chunkQuerySpec.nextFragment.get()) { + ChunkQuerySpec const* sPtr = &chunkQuerySpec; + while (sPtr) { + LOGS(_log, LOG_LVL_TRACE, "nextFragment"); + for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { + LOGS(_log, LOG_LVL_DEBUG, __func__ << " q=" << (sPtr->queries).at(t)); + } + for (auto const& sbi : sPtr->subChunkIds) { + LOGS(_log, LOG_LVL_DEBUG, __func__ << " sbi=" << sbi); + } + // Linked fragments will not have valid subChunkTables vectors, + // So, we reuse the root fragment's vector. + _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, + sPtr->queries); + sPtr = sPtr->nextFragment.get(); + } + } else { + LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); + for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { + LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); + } + _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, + chunkQuerySpec.queries); + } + */ + nlohmann::json jsFrag = {{"resultTable", mInfo.resultName}, + {"queries", nlohmann::json::array()}, + {"subchunkTables", nlohmann::json::array()}, + {"subchunkIds", nlohmann::json::array()}}; + + auto& jsQueries = jsFrag["queries"]; + nlohmann::json jsQry = {{"subQuery", mInfo.qry}}; + jsQueries.push_back(move(jsQry)); + + jsFragments.push_back(move(jsFrag)); + + return jsJobMsgPtr; + } + MySqlConfig newMySqlConfig() { string user = "qsmaster"; string password = ""; @@ -150,6 +251,7 @@ struct Fixture { BOOST_FIXTURE_TEST_SUITE(Basic, Fixture, *boost::unit_test::timeout(20)) BOOST_AUTO_TEST_CASE(Simple) { + /* &&& WorkerConfig::create(); MsgInfo mInfo; auto msgJson = newTaskJson(mInfo); @@ -174,9 +276,35 @@ BOOST_AUTO_TEST_CASE(Simple) { Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); + */ + + WorkerConfig::create(); + MsgInfo mInfo; + auto msgJson = newTaskJson(mInfo); + shared_ptr sendC(SendChannel::newNopChannel()); + auto sc = FileChannelShared::create(sendC, mInfo.czarId); + FakeBackend::Ptr backend = make_shared(); + shared_ptr crm = ChunkResourceMgr::newMgr(backend); + SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); + auto const queries = queriesAndChunks(); + auto ujData = lsst::qserv::wbase::UberJobData::create(mInfo.uberJobId, mInfo.czarName, mInfo.czarId, + mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + lsst::qserv::proto::ScanInfo scanInfo; + scanInfo.scanRating = mInfo.scanRating; + scanInfo.infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + vector taskVect = Task::createTasksForChunk( + ujData, *msgJson, sc, scanInfo, + mInfo.scanInteractive, mInfo.maxTableSize, + crm, + newMySqlConfig(), sqlConnMgr, + queries); + Task::Ptr task = taskVect[0]; + QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); + BOOST_CHECK(a->runQuery()); } BOOST_AUTO_TEST_CASE(Output) { + /* &&& WorkerConfig::create(); string out; MsgInfo mInfo; @@ -200,6 +328,32 @@ BOOST_AUTO_TEST_CASE(Output) { Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); + */ + WorkerConfig::create(); + string out; + MsgInfo mInfo; + auto msgJson = newTaskJson(mInfo); + shared_ptr sendC(SendChannel::newStringChannel(out)); + auto sc = FileChannelShared::create(sendC, mInfo.czarId); + FakeBackend::Ptr backend = make_shared(); + shared_ptr crm = ChunkResourceMgr::newMgr(backend); + SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); + auto const queries = queriesAndChunks(); + auto ujData = lsst::qserv::wbase::UberJobData::create(mInfo.uberJobId, mInfo.czarName, mInfo.czarId, + mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + lsst::qserv::proto::ScanInfo scanInfo; + scanInfo.scanRating = mInfo.scanRating; + scanInfo.infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + vector taskVect = Task::createTasksForChunk( + ujData, *msgJson, sc, scanInfo, + mInfo.scanInteractive, mInfo.maxTableSize, + crm, + newMySqlConfig(), sqlConnMgr, + queries); + Task::Ptr task = taskVect[0]; + QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); + BOOST_CHECK(a->runQuery()); + } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wdb/testQuerySql.cc b/src/wdb/testQuerySql.cc new file mode 100644 index 0000000000..f28d733d65 --- /dev/null +++ b/src/wdb/testQuerySql.cc @@ -0,0 +1,101 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2013-2015 AURA/LSST. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +/** + * @brief Simple testing for class QuerySql + * + * @author Daniel L. Wang, SLAC + */ + +// Third-party headers + +// Qserv headers +#include "proto/worker.pb.h" +#include "wdb/QuerySql.h" +#include "wdb/QuerySql_Batch.h" + +// Boost unit test header +#define BOOST_TEST_MODULE QuerySql_1 +#include + +namespace test = boost::test_tools; + +//&&& delete file + +using lsst::qserv::proto::TaskMsg_Fragment; +using lsst::qserv::proto::TaskMsg_Subchunk; +using lsst::qserv::wdb::QuerySql; + +struct Fixture { + Fixture() { + defaultDb = "Winter"; + defaultResult = "myResult"; + } + ~Fixture() {} + + TaskMsg_Fragment makeFragment() { + TaskMsg_Fragment f; + // "Real" subchunk query text should include + // pre-substituted subchunk query text. + f.add_query("SELECT o1.*, o2.* FROM Object_1001 o1, Object_1001 o2;"); + f.set_resulttable("fragResult"); + TaskMsg_Subchunk sc; + sc.set_database("obsolete"); + lsst::qserv::proto::TaskMsg_Subchunk_DbTbl* dbTbl = sc.add_dbtbl(); + dbTbl->set_db(defaultDb); + dbTbl->set_tbl("Object"); + sc.add_id(1111); + sc.add_id(1222); + f.mutable_subchunks()->CopyFrom(sc); + return f; + } + + void printQsql(QuerySql const& q) { std::cout << "qsql=" << q << std::endl; } + std::string defaultDb; + std::string defaultResult; +}; + +BOOST_FIXTURE_TEST_SUITE(QuerySqlSuite, Fixture) + +BOOST_AUTO_TEST_CASE(Basic) { + std::shared_ptr qSql; + TaskMsg_Fragment frag = makeFragment(); + qSql = std::make_shared(defaultDb, 1001, frag, true, defaultResult); + BOOST_CHECK(qSql.get()); + printQsql(*qSql); +} + +BOOST_AUTO_TEST_CASE(QueryBatch) { + std::shared_ptr qSql; + TaskMsg_Fragment frag = makeFragment(); + qSql = std::make_shared(defaultDb, 1001, frag, true, defaultResult); + BOOST_CHECK(qSql.get()); + + QuerySql::Batch build("QueryBuildSub", qSql->buildList); + QuerySql::Batch& batch = build; + while (!batch.isDone()) { + std::string piece = batch.current(); + batch.next(); + } +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index cafbfb8dc2..af565ad9c1 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -54,7 +54,7 @@ void ChunkTasksQueue::queueTask(std::vector const& tasks) { // Correct ChunkTask wasn't found, make a new one. std::pair ele(chunkId, std::make_shared(chunkId)); auto res = _chunkMap.insert(ele); // insert should fail if the key already exists. - LOGS(_log, LOG_LVL_TRACE, " queueTask chunk=" << chunkId << " created=" << res.second); + LOGS(_log, LOG_LVL_DEBUG, " queueTask chunk=" << chunkId << " created=" << res.second); iter = res.first; } } diff --git a/src/wsched/testSchedulers.cc b/src/wsched/testSchedulers.cc index f2217b1b68..17ff5523c9 100644 --- a/src/wsched/testSchedulers.cc +++ b/src/wsched/testSchedulers.cc @@ -303,6 +303,7 @@ BOOST_AUTO_TEST_CASE(Grouping) { } BOOST_AUTO_TEST_CASE(GroupMaxThread) { + #if 0 // &&& fix and re-enable // Test that maxThreads is meaningful. LOGS(_log, LOG_LVL_WARN, "Test_case GroupMaxThread"); diff --git a/src/xrdreq/QueryManagementAction.h b/src/xrdreq/QueryManagementAction.h new file mode 100644 index 0000000000..ec5ff91588 --- /dev/null +++ b/src/xrdreq/QueryManagementAction.h @@ -0,0 +1,97 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H +#define LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H + +// System headers +#include +#include +#include +#include +#include + +// Qserv headers +#include "global/intTypes.h" +#include "proto/worker.pb.h" + +namespace lsst::qserv::xrdreq { + +/** + * Class QueryManagementAction is an interface for managing query completion/cancellation + * at all Qserv workers that are connected as "publishers" to the XROOTD redirector. + */ +// &&& need to get the same functionality using json messages, and not in xrdreq. +class QueryManagementAction : public std::enable_shared_from_this { +public: + /// The reponse type represents errors reported by the workers, where worker + /// names are the keys. And the values are the error messages. Empty strings + /// indicate the succesful completion of the requests. + using Response = std::map; + + /// The callback function type to be used for notifications on the operation completion. + using CallbackType = std::function; + + /** + * The front-end method for initiating the operation at all workers. + * + * @note The only way to track the completion of the requests sent via + * this interface is by providing the callback function. The request delivery + * is not guaranteeded in case if the XROOTD/SSI network will be clogged by + * the heavy traffic. It's safe to call the same operation many times if needed. + * + * @param xrootdFrontendUrl A location of the XROOTD redirector. + * @param op An operation be initiated at the workers. + * @param onFinish The optional callback to be fired upon the completion of + * the requested operation. + * + * @throws std::runtime_error For failures encountered when connecting to + * the manager or initiating the requesed operation. + */ + static void notifyAllWorkers(std::string const& xrootdFrontendUrl, proto::QueryManagement::Operation op, + uint32_t czarId, QueryId queryId, CallbackType onFinish = nullptr); + + QueryManagementAction(QueryManagementAction const&) = delete; + QueryManagementAction& operator=(QueryManagementAction const&) = delete; + virtual ~QueryManagementAction(); + +private: + QueryManagementAction(); + + /** + * The actual implementation of the request processor. + * @see QueryManagementAction::notifyAllWorkers() + */ + void _notifyAllWorkers(std::string const& xrootdFrontendUrl, proto::QueryManagement::Operation op, + uint32_t czarId, QueryId queryId, CallbackType onFinish); + + /// The collection of worker responses. + Response _response; + + /// The counter will get incremented as worker responses will be received. + /// User-provided callback function (if any) will be called when all requests + /// will finish (succeed or fail). + std::atomic _numWorkerRequestsFinished{0}; +}; + +} // namespace lsst::qserv::xrdreq + +#endif // LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H diff --git a/src/xrdsvc/ChannelStream.h b/src/xrdsvc/ChannelStream.h new file mode 100644 index 0000000000..61c8777e7e --- /dev/null +++ b/src/xrdsvc/ChannelStream.h @@ -0,0 +1,75 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2014-2018 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_XRDSVC_CHANNELSTREAM_H +#define LSST_QSERV_XRDSVC_CHANNELSTREAM_H + +// System headers +#include +#include +#include +#include + +// qserv headers +#include "xrdsvc/StreamBuffer.h" + +// Third-party headers +#include "XrdSsi/XrdSsiErrInfo.hh" // required by XrdSsiStream +#include "XrdSsi/XrdSsiStream.hh" + +namespace lsst::qserv::xrdsvc { + +/// ChannelStream is an implementation of an XrdSsiStream that accepts +/// SendChannel streamed data. +class ChannelStream : public XrdSsiStream { // &&& delete +public: + ChannelStream(); + virtual ~ChannelStream(); + + /// Push in a data packet + void append(StreamBuffer::Ptr const &StreamBuffer, bool last); + + /// Empty _msgs, calling StreamBuffer::Recycle() where needed. + void clearMsgs(); + + /// Pull out a data packet as a Buffer object (called by XrdSsi code) + Buffer *GetBuff(XrdSsiErrInfo &eInfo, int &dlen, bool &last) override; + + bool closed() const { return _closed; } + + uint64_t getSeq() const { return _seq; } + +private: + bool _closed; ///< Closed to new append() calls? + // Can keep a deque of (buf, bufsize) to reduce copying, if needed. + std::deque _msgs; ///< Message queue + std::mutex _mutex; ///< _msgs protection + std::condition_variable _hasDataCondition; ///< _msgs condition + uint64_t const _seq; ///< Unique identifier for this instance. + static std::atomic _sequenceSource; ///< Source of unique identifiers. + std::atomic _appendCount{0}; ///< number of appends + std::atomic _getBufCount{0}; ///< number of buffers +}; + +} // namespace lsst::qserv::xrdsvc + +#endif // LSST_QSERV_XRDSVC_CHANNELSTREAM_H diff --git a/src/xrdsvc/HttpSvc.cc b/src/xrdsvc/HttpSvc.cc index 6d1df4d0d0..365f4c0a8b 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/xrdsvc/HttpSvc.cc @@ -147,6 +147,12 @@ uint16_t HttpSvc::start() { HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "/querystatus", http::AuthType::REQUIRED); }}}); + _httpServerPtr->addHandlers( + {{"POST", "/querystatus", + [self](shared_ptr const& req, shared_ptr const& resp) { + HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "QUERYSTATUS", + http::AuthType::REQUIRED); + }}}); _httpServerPtr->start(); // Initialize the I/O context and start the service threads. At this point diff --git a/src/xrdsvc/SsiRequest.cc b/src/xrdsvc/SsiRequest.cc index 19db57205c..1b4ca9aeb5 100644 --- a/src/xrdsvc/SsiRequest.cc +++ b/src/xrdsvc/SsiRequest.cc @@ -90,17 +90,9 @@ uint64_t countLimiter = 0; // LockupDB // Step 4 /// Called by XrdSsi to actually process a request. void SsiRequest::execute(XrdSsiRequest& req) { - LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute start"); util::Timer t; LOGS(_log, LOG_LVL_DEBUG, "Execute request, resource=" << _resourceName); - char* reqData = nullptr; - int reqSize; - t.start(); - reqData = req.GetRequest(reqSize); - t.stop(); - LOGS(_log, LOG_LVL_DEBUG, "GetRequest took " << t.getElapsed() << " seconds"); - // We bind this object to the request now. This allows us to respond at any // time (much simpler). Though the manual forgot to say that all pending // events will be reflected on a different thread the moment we bind the @@ -123,73 +115,20 @@ void SsiRequest::execute(XrdSsiRequest& req) { // Process the request switch (ru.unitType()) { - case ResourceUnit::DBCHUNK: { - LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute DBCHUNK"); + case ResourceUnit::DBCHUNK: { // &&& delete // Increment the counter of the database/chunk resources in use - _foreman->resourceMonitor()->increment(_resourceName); - - // reqData has the entire request, so we can unpack it without waiting for - // more data. - LOGS(_log, LOG_LVL_DEBUG, "Decoding TaskMsg of size " << reqSize); - auto taskMsg = std::make_shared(); - if (!taskMsg->ParseFromArray(reqData, reqSize) || !taskMsg->IsInitialized()) { - reportError("Failed to decode TaskMsg on resource db=" + ru.db() + - " chunkId=" + std::to_string(ru.chunk())); - return; - } - - QSERV_LOGCONTEXT_QUERY_JOB(taskMsg->queryid(), taskMsg->jobid()); - - if (!taskMsg->has_db() || !taskMsg->has_chunkid() || (ru.db() != taskMsg->db()) || - (ru.chunk() != taskMsg->chunkid())) { - reportError("Mismatched db/chunk in TaskMsg on resource db=" + ru.db() + - " chunkId=" + std::to_string(ru.chunk())); - return; - } - - if (not(taskMsg->has_queryid() && taskMsg->has_jobid() && taskMsg->has_scaninteractive() && - taskMsg->has_attemptcount() && taskMsg->has_czarid())) { - reportError(std::string("taskMsg missing required field ") + - " queryid:" + std::to_string(taskMsg->has_queryid()) + - " jobid:" + std::to_string(taskMsg->has_jobid()) + - " scaninteractive:" + std::to_string(taskMsg->has_scaninteractive()) + - " attemptcount:" + std::to_string(taskMsg->has_attemptcount()) + - " czarid:" + std::to_string(taskMsg->has_czarid())); - return; - } - switch (wconfig::WorkerConfig::instance()->resultDeliveryProtocol()) { - case wconfig::ConfigValResultDeliveryProtocol::XROOT: - case wconfig::ConfigValResultDeliveryProtocol::HTTP: - _channelShared = wbase::FileChannelShared::create(sendChannel, taskMsg->czarid(), - _foreman->chunkInventory()->id()); - break; - default: - throw std::runtime_error("SsiRequest::" + std::string(__func__) + - " unsupported result delivery protocol"); - } - auto const tasks = wbase::Task::createTasks(taskMsg, _channelShared, _foreman->chunkResourceMgr(), - _foreman->mySqlConfig(), _foreman->sqlConnMgr(), - _foreman->queriesAndChunks(), _foreman->httpPort()); - for (auto const& task : tasks) { - _tasks.push_back(task); - } + _foreman->resourceMonitor()->increment(_resourceName); // &&& TODO:UJ make sure this is implemented elsewhere. - // Now that the request is decoded (successfully or not), release the - // xrootd request buffer. To avoid data races, this must happen before - // the task is handed off to another thread for processing, as there is a - // reference to this SsiRequest inside the reply channel for the task, - // and after the call to BindRequest. - ReleaseRequestBuffer(); - t.start(); - _foreman->processTasks(tasks); // Queues tasks to be run later. - t.stop(); - LOGS(_log, LOG_LVL_DEBUG, - "Enqueued TaskMsg for " << ru << " in " << t.getElapsed() << " seconds"); + reportError("&&& DBCHUNK requests are no longer available resource db=" + ru.db() + + " chunkId=" + std::to_string(ru.chunk())); break; } - case ResourceUnit::QUERY: { - LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute QUERY"); + case ResourceUnit::QUERY: { // &&& delete LOGS(_log, LOG_LVL_DEBUG, "Parsing request details for resource=" << _resourceName); + + reportError("&&& QUERY requests are no longer available"); + + /* &&& proto::QueryManagement request; try { // reqData has the entire request, so we can unpack it without waiting for @@ -239,17 +178,16 @@ void SsiRequest::execute(XrdSsiRequest& req) { // Send back the empty response since no info is expected by a caller // for this type of requests beyond the usual error notifications (if any). this->reply((char const*)0, 0); + */ break; } default: - LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute default"); reportError("Unexpected unit type '" + std::to_string(ru.unitType()) + "', resource name: " + _resourceName); - break; } - // Note that upon exit the _finMutex will be unlocked allowing Finished() // to actually do something once everything is actually setup. + } /// Called by SSI to free resources. diff --git a/src/xrdsvc/SsiRequest.h b/src/xrdsvc/SsiRequest.h new file mode 100644 index 0000000000..5850d18bf8 --- /dev/null +++ b/src/xrdsvc/SsiRequest.h @@ -0,0 +1,128 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2015 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_XRDSVC_SSIREQUEST_H +#define LSST_QSERV_XRDSVC_SSIREQUEST_H + +// System headers +#include +#include +#include +#include + +// Third-party headers +#include "XrdSsi/XrdSsiResponder.hh" + +// Qserv headers +#include "global/ResourceUnit.h" +#include "mysql/MySqlConfig.h" +#include "xrdsvc/StreamBuffer.h" + +// Forward declarations +class XrdSsiService; + +namespace lsst::qserv { +namespace wbase { +class FileChannelShared; +class SendChannel; +class Task; +} // namespace wbase +namespace wcontrol { +class Foreman; +} +} // namespace lsst::qserv + +namespace lsst::qserv::xrdsvc { + +class ChannelStream; +class StreamBuffer; + +/// An implementation of XrdSsiResponder that is used by SsiService to provide +/// qserv worker services. The SSI interface encourages such an approach, and +/// object lifetimes are explicitly stated in the documentation which we +/// adhere to using BindRequest() and UnBindRequest() responder methods. +class SsiRequest : public XrdSsiResponder, public std::enable_shared_from_this { // &&& delete if possible +public: + // Smart pointer definitions + + typedef std::shared_ptr ValidatorPtr; + typedef std::shared_ptr Ptr; + + /// Use factory to ensure proper construction for enable_shared_from_this. + static SsiRequest::Ptr newSsiRequest(std::string const& rname, + std::shared_ptr const& processor); + + virtual ~SsiRequest(); + + void execute(XrdSsiRequest& req); + + /** + * Implements the virtual method defined in the base class + * @see XrdSsiResponder::Finished + */ + void Finished(XrdSsiRequest& req, XrdSsiRespInfo const& rinfo, bool cancel = false) override; + + bool isFinished() { return _reqFinished; } + + bool reply(char const* buf, int bufLen); + bool replyError(std::string const& msg, int code); + bool replyStream(StreamBuffer::Ptr const& sbuf, bool last); + + bool sendMetadata(const char* buf, int blen); + + /// Call this to allow object to die after it truly is no longer needed. + /// i.e. It is know Finish() will not be called. + /// NOTE: It is important that any non-static SsiRequest member + /// function make a local copy of the returned pointer so that + /// SsiRequest is guaranteed to live to the end of + /// the function call. + Ptr freeSelfKeepAlive(); + + uint64_t getSeq() const; + +private: + /// Constructor (called by the static factory method newSsiRequest) + SsiRequest(std::string const& rname, std::shared_ptr const& processor); + + /// For internal error reporting + void reportError(std::string const& errStr); + +private: + ValidatorPtr _validator; ///< validates request against what's available + std::shared_ptr const _foreman; ///< actual msg processor + + std::mutex _finMutex; ///< Protects execute() from Finish(), _finished, and _stream + std::atomic _reqFinished{false}; ///< set to true when Finished called + std::string _resourceName; ///< chunk identifier + + std::shared_ptr _stream; + std::shared_ptr _channelShared; ///< Must live before Finished() gets called. + std::vector> _tasks; ///< List of tasks for use in cancellation. + + /// Make sure this object exists until Finish() is called. + /// Make a local copy before calling reset() within and non-static member function. + Ptr _selfKeepAlive; +}; + +} // namespace lsst::qserv::xrdsvc + +#endif // LSST_QSERV_XRDSVC_SSIREQUEST_H From 8f6626405499cc6a576cb453c73aef47c9e14ef4 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 3 Sep 2024 11:52:38 -0700 Subject: [PATCH 06/15] Added cancellation code and for queries, uberjobs, and czar restart. --- src/CMakeLists.txt | 1 - src/ccontrol/UserQuerySelect.cc | 8 +- src/czar/ActiveWorker.cc | 359 ++++++++++++--------------- src/czar/Czar.cc | 7 +- src/czar/CzarChunkMap.cc | 4 +- src/czar/CzarRegistry.cc | 13 + src/czar/CzarRegistry.h | 10 + src/http/WorkerQueryStatusData.cc | 376 ++++++++++++++++++----------- src/http/WorkerQueryStatusData.h | 321 +++++++++++++++++------- src/http/testStatusData.cc | 83 ++++--- src/proto/worker.proto | 1 - src/protojson/ScanTableInfo.h | 2 +- src/qdisp/Executive.cc | 54 +++-- src/qdisp/Executive.h | 2 +- src/qdisp/JobBase.h | 5 +- src/qdisp/JobQuery.cc | 4 +- src/qdisp/SharedResources.h | 64 +++++ src/qdisp/UberJob.cc | 42 +++- src/util/xrootd.cc | 91 +++++++ src/util/xrootd.h | 44 ++++ src/wbase/MsgProcessor.h | 64 ----- src/wbase/Task.cc | 25 +- src/wbase/Task.h | 2 +- src/wbase/UberJobData.cc | 104 +++++++- src/wbase/UberJobData.h | 11 + src/wcontrol/Foreman.cc | 2 + src/wdb/testQueryRunner.cc | 32 +-- src/wpublish/QueriesAndChunks.h | 7 + src/wpublish/QueryStatistics.cc | 7 + src/wpublish/QueryStatistics.h | 2 + src/xrdreq/QueryManagementAction.h | 97 -------- src/xrdsvc/ChannelStream.h | 75 ------ src/xrdsvc/HttpSvc.cc | 2 +- src/xrdsvc/HttpWorkerCzarModule.cc | 7 + src/xrdsvc/SsiRequest.cc | 348 -------------------------- src/xrdsvc/SsiRequest.h | 128 ---------- 36 files changed, 1170 insertions(+), 1234 deletions(-) create mode 100644 src/qdisp/SharedResources.h create mode 100644 src/util/xrootd.cc create mode 100644 src/util/xrootd.h delete mode 100644 src/wbase/MsgProcessor.h delete mode 100644 src/xrdreq/QueryManagementAction.h delete mode 100644 src/xrdsvc/ChannelStream.h delete mode 100644 src/xrdsvc/SsiRequest.cc delete mode 100644 src/xrdsvc/SsiRequest.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 822a8ae3b5..4b53c9ec46 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -89,4 +89,3 @@ add_subdirectory(www) add_subdirectory(xrdlog) add_subdirectory(xrdsvc) - diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index beaec52717..22d23719c1 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -529,6 +529,7 @@ void UserQuerySelect::buildAndSendUberJobs() { } void UserQuerySelect::buildAndSendUberJobs() { + // &&& NEED CODE - this function should check if the worker is DEAD. TODO:UJ string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); LOGS(_log, LOG_LVL_DEBUG, funcN << " start"); @@ -566,8 +567,6 @@ void UserQuerySelect::buildAndSendUberJobs() { // Make a map of all jobs in the executive. // TODO:UJ Maybe a check should be made that all databases are in the same family? - - // keep cycling through workers until no more chunks to place. // - create a map of UberJobs key=, val=> // - for chunkId in `unassignedChunksInQuery` @@ -656,8 +655,9 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_ERROR, errStr); } - // Add worker contact info to UberJobs. - auto const wContactMap = czRegistry->getWorkerContactMap(); + // Add worker contact info to UberJobs. The czar can't do anything without + // the contact map, so it will wait. This should only ever be an issue at startup. + auto const wContactMap = czRegistry->waitForWorkerContactMap(); LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); for (auto const& [wIdKey, ujVect] : workerJobMap) { auto iter = wContactMap->find(wIdKey); diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index b6454cd83a..2325b014c8 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -238,230 +238,138 @@ string ActiveWorker::_dump() const { string ActiveWorker::getStateStr(State st) { switch (st) { - case ALIVE: return string("ALIVE"); - case QUESTIONABLE: return string("QUESTIONABLE"); - case DEAD: return string("DEAD"); + case ALIVE: + return string("ALIVE"); + case QUESTIONABLE: + return string("QUESTIONABLE"); + case DEAD: + return string("DEAD"); } return string("unknown"); } bool ActiveWorker::compareContactInfo(http::WorkerContactInfo const& wcInfo) const { lock_guard lg(_aMtx); - return _wqsData->_wInfo->isSameContactInfo(wcInfo); + auto wInfo_ = _wqsData->getWInfo(); + if (wInfo_ == nullptr) return false; + return wInfo_->isSameContactInfo(wcInfo); } void ActiveWorker::setWorkerContactInfo(http::WorkerContactInfo::Ptr const& wcInfo) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " new info=" << wcInfo->dump()); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " new info=" << wcInfo->dump()); lock_guard lg(_aMtx); - _wqsData->_wInfo = wcInfo; + _wqsData->setWInfo(wcInfo); } void ActiveWorker::_changeStateTo(State newState, double secsSinceUpdate, string const& note) { auto lLvl = (newState == DEAD) ? LOG_LVL_ERROR : LOG_LVL_INFO; - LOGS(_log, lLvl, note << " oldState=" << getStateStr(_state) << " newState=" << getStateStr(newState) << " secsSince=" << secsSinceUpdate); + LOGS(_log, lLvl, + note << " oldState=" << getStateStr(_state) << " newState=" << getStateStr(newState) + << " secsSince=" << secsSinceUpdate); _state = newState; } -void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime) { - // &&& function too long - lock_guard lg(_aMtx); - double secsSinceUpdate = _wqsData->_wInfo->timeSinceRegUpdateSeconds(); - // Update the last time the registry contacted this worker. - switch (_state) { - case ALIVE: { - if (secsSinceUpdate > timeoutAliveSecs) { - _changeStateTo(QUESTIONABLE, secsSinceUpdate, cName(__func__)); - // Anything that should be done here? - } - break; - } - case QUESTIONABLE: { - if (secsSinceUpdate < timeoutAliveSecs) { - _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); - } - if (secsSinceUpdate > timeoutDeadSecs) { - _changeStateTo(DEAD, secsSinceUpdate, cName(__func__)); - // &&& TODO:UJ all uberjobs for this worker need to die. - } - break; - } - case DEAD: { - LOGS(_log, LOG_LVL_ERROR, "&&& NEED CODE"); - if (secsSinceUpdate < timeoutAliveSecs) { - _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); - } else { - // Don't waste time on this worker until the registry has heard from it. +void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, + double maxLifetime) { + bool newlyDeadWorker = false; + http::WorkerContactInfo::Ptr wInfo_; + { + lock_guard lg(_aMtx); + wInfo_ = _wqsData->getWInfo(); + if (wInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no WorkerContactInfo"); return; } - break; - } + double secsSinceUpdate = (wInfo_ == nullptr) ? timeoutDeadSecs : wInfo_->timeSinceRegUpdateSeconds(); - } - - // Check how many messages are currently being sent to the worker, if at the limit, return - if (_wqsData->_qIdDoneKeepFiles.empty() && _wqsData->_qIdDoneDeleteFiles.empty() && _wqsData->_qIdDeadUberJobs.empty()) { - return; - } - int tCount = _conThreadCount; - if (tCount > _maxConThreadCount) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " not sending message since at max threads " << tCount); - return; - } - - // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a - // message to send to the worker. -#if 0 // &&& - auto now = CLOCK::now(); - auto const czarConfig = cconfig::CzarConfig::instance(); - - shared_ptr jsWorkerReqPtr = make_shared(); - json& jsWorkerR = *jsWorkerReqPtr; - jsWorkerR["version"] = http::MetaModule::version; - jsWorkerR["instance_id"] = czarConfig->replicationInstanceId(); - jsWorkerR["auth_key"] = czarConfig->replicationAuthKey(); - jsWorkerR["worker"] = _wInfo->wId; - jsWorkerR["qiddonekeepfiles"] = json::array(); - jsWorkerR["qiddonedeletefiles"] = json::array(); - jsWorkerR["qiddeaduberjobs"] = json::array(); - jsWorkerR["czar"] = json::object(); - auto& jsWCzar = jsWorkerR["czar"]; - jsWCzar["name"] = czarConfig->name(); - jsWCzar["id"]= czarConfig->id(); - jsWCzar["management-port"] = czarConfig->replicationHttpPort(); - jsWCzar["management-host-name"] = util::get_current_host_fqdn(); - - - { - auto& jsDoneKeep = jsWorkerR["qiddonekeepfiles"]; - auto iterDoneKeep = _qIdDoneKeepFiles.begin(); - while (iterDoneKeep != _qIdDoneKeepFiles.end()) { - auto qId = iterDoneKeep->first; - jsDoneKeep.push_back(qId); - auto tmStamp = iterDoneKeep->second; - double ageSecs = std::chrono::duration(now - tmStamp).count(); - if (ageSecs > maxLifetime) { - iterDoneKeep = _qIdDoneKeepFiles.erase(iterDoneKeep); - } else { - ++iterDoneKeep; - } - } - } - { - auto& jsDoneDelete = jsWorkerR["qiddonedeletefiles"]; - auto iterDoneDelete = _qIdDoneDeleteFiles.begin(); - while (iterDoneDelete != _qIdDoneDeleteFiles.end()) { - auto qId = iterDoneDelete->first; - jsDoneDelete.push_back(qId); - auto tmStamp = iterDoneDelete->second; - double ageSecs = std::chrono::duration(now - tmStamp).count(); - if (ageSecs > maxLifetime) { - iterDoneDelete = _qIdDoneDeleteFiles.erase(iterDoneDelete); - } else { - ++iterDoneDelete; + // Update the last time the registry contacted this worker. + switch (_state) { + case ALIVE: { + if (secsSinceUpdate >= timeoutAliveSecs) { + _changeStateTo(QUESTIONABLE, secsSinceUpdate, cName(__func__)); + // &&& Anything else that should be done here? + } + break; } - } - } - { - auto& jsDeadUj = jsWorkerR["qiddeaduberjobs"]; - auto iterDeadUjQid = _qIdDeadUberJobs.begin(); - while (iterDeadUjQid != _qIdDeadUberJobs.end()) { - TIMEPOINT oldestTm; // default is zero - auto qId = iterDeadUjQid->first; - auto& ujIdMap = iterDeadUjQid->second; - - json jsQidUj = {{"qid", qId}, {"ujids", json::array()}}; - auto& jsUjIds = jsQidUj["ujids"]; - - auto iterUjId = ujIdMap.begin(); - bool addedUjId = false; - while (iterUjId != ujIdMap.end()) { - UberJobId ujId = iterUjId->first; - auto tmStamp = iterUjId->second; - if (tmStamp > oldestTm) { - oldestTm = tmStamp; + case QUESTIONABLE: { + if (secsSinceUpdate < timeoutAliveSecs) { + _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); } - - jsUjIds.push_back(ujId); - addedUjId = true; - double ageSecs = std::chrono::duration(now - tmStamp).count(); - if (ageSecs > maxLifetime) { - iterUjId = ujIdMap.erase(iterUjId); - } else { - ++iterUjId; + if (secsSinceUpdate >= timeoutDeadSecs) { + _changeStateTo(DEAD, secsSinceUpdate, cName(__func__)); + // All uberjobs for this worker need to die. + newlyDeadWorker = true; } + break; } - - if (addedUjId) { - jsDeadUj.push_back(jsQidUj); - } - - if (ujIdMap.empty() - || std::chrono::duration(now - oldestTm).count() > maxLifetime) { - iterDeadUjQid = _qIdDeadUberJobs.erase(iterDeadUjQid); - } else { - ++iterDeadUjQid; + case DEAD: { + if (secsSinceUpdate < timeoutAliveSecs) { + _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); + } else { + // Don't waste time on this worker until the registry has heard from it. + // &&& If it's been a really really long time, maybe delete this entry ??? + return; + } + break; } } } -#endif // &&& - - auto jsWorkerReqPtr = _wqsData->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); - - // Start a thread to send the message. (Maybe these should go on the qdisppool? &&&) - // put this in a different function and start the thread.&&&; - _sendStatusMsg(jsWorkerReqPtr); -} - -#if 0 // &&& -bool ActiveWorker::_parse(nlohmann::json const& jsWorkerReq) { - auto const czarConfig = cconfig::CzarConfig::instance(); - http::RequestBodyJSON rbWReq(jsWorkerReq); - if (jsWorkerReq["version"] != http::MetaModule::version) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " bad version"); - return false; + // _aMtx must not be held when calling this. + if (newlyDeadWorker) { + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " worker " << wInfo_->wId << " appears to have died, reassigning its jobs."); + czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInfo_->wId); } + shared_ptr jsWorkerReqPtr; + { + lock_guard lg(_aMtx); //&&& needed ??? + lock_guard mapLg(_wqsData->mapMtx); + // Check how many messages are currently being sent to the worker, if at the limit, return + if (_wqsData->qIdDoneKeepFiles.empty() && _wqsData->qIdDoneDeleteFiles.empty() && + _wqsData->qIdDeadUberJobs.empty()) { + return; + } + int tCount = _conThreadCount; + if (tCount > _maxConThreadCount) { + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " not sending message since at max threads " << tCount); + return; + } - http::RequestBodyJSON rbCzar(rbWReq.required("czar")); - auto czarName = rbCzar.required("name"); - auto czarId = rbCzar.required("id"); - auto czarPort = rbCzar.required("management-port"); - auto czarHostName = rbCzar.required("management-host-name"); - /* &&& - jsWorkerReq["instance_id"] != czarConfig->replicationInstanceId(); - jsWorkerReq["auth_key"] != czarConfig->replicationAuthKey(); - jsWorkerReq["worker"] != _wInfo->wId; - auto& jsWCzar = jsWorkerReq["czar"]; - jsWCzar["name"] != czarConfig->name(); - jsWCzar["id"] != czarConfig->id(); - jsWCzar["management-port"] != czarConfig->replicationHttpPort(); - jsWCzar["management-host-name"] != util::get_current_host_fqdn(); - */ - - - auto& jsQIdDoneKeepFiles = jsWorkerReq["qiddonekeepfiles"]; - for (auto const& qidKeep : jsQIdDoneKeepFiles) { - + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // message to send to the worker. + jsWorkerReqPtr = _wqsData->serializeJson(maxLifetime); } - auto& jsQIdDoneDeleteFiles = jsWorkerReq["qiddonedeletefiles"]; - - auto& jsQIdDeadUberJobs = jsWorkerReq["qiddeaduberjobs"]; + // &&& Maybe only send the status message if the lists are not empty ??? + // Start a thread to send the message. (Maybe these should go on the qdisppool? &&&) + // put this in a different function and start the thread.&&&; + //&&& _sendStatusMsg(wInfo_, jsWorkerReqPtr); + Ptr thisPtr = shared_from_this(); + auto sendStatusMsgFunc = [thisPtr, wInfo_, jsWorkerReqPtr](util::CmdData*) { + thisPtr->_sendStatusMsg(wInfo_, jsWorkerReqPtr); + }; + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(sendStatusMsgFunc)); + auto qdisppool = czar::Czar::getCzar()->getQdispPool(); + qdisppool->queCmd(cmd, 1); } -#endif // &&& - -void ActiveWorker::_sendStatusMsg(std::shared_ptr const& jsWorkerReqPtr) { +void ActiveWorker::_sendStatusMsg(http::WorkerContactInfo::Ptr const& wInf, + std::shared_ptr const& jsWorkerReqPtr) { auto& jsWorkerReq = *jsWorkerReqPtr; auto const method = http::Method::POST; - auto const& wInf = _wqsData->_wInfo; - string const url = "http://" + wInf->wHost + ":" + to_string(wInf->wPort) + "/querystatus"; + if (wInf == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo was null."); + return; + } + auto [ciwId, ciwHost, ciwManag, ciwPort] = wInf->getAll(); + string const url = "http://" + ciwHost + ":" + to_string(ciwPort) + "/querystatus"; vector const headers = {"Content-Type: application/json"}; auto const& czarConfig = cconfig::CzarConfig::instance(); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << jsWorkerReq); string const requestContext = "Czar: '" + http::method2string(method) + "' stat request to '" + url + "'"; LOGS(_log, LOG_LVL_TRACE, @@ -473,7 +381,13 @@ void ActiveWorker::_sendStatusMsg(std::shared_ptr const& jsWorke try { json const response = client.readAsJson(); if (0 != response.at("success").get()) { - transmitSuccess = true; + bool startupTimeChanged = false; + tie(transmitSuccess, startupTimeChanged) = _wqsData->handleResponseJson(response); + if (startupTimeChanged) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " worker startupTime changed, likely rebooted."); + // kill all incomplete UberJobs on this worker. + czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInf->wId); + } } else { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " response success=0"); } @@ -483,12 +397,19 @@ void ActiveWorker::_sendStatusMsg(std::shared_ptr const& jsWorke } if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure"); - } else { - // parse the return statement and remove the indicated entries from the list - //HERE &&&; } } +void ActiveWorker::addToDoneDeleteFiles(QueryId qId) { _wqsData->addToDoneDeleteFiles(qId); } + +void ActiveWorker::addToDoneKeepFiles(QueryId qId) { _wqsData->addToDoneKeepFiles(qId); } + +void ActiveWorker::removeDeadUberJobsFor(QueryId qId) { _wqsData->removeDeadUberJobsFor(qId); } + +void ActiveWorker::addDeadUberJob(QueryId qId, UberJobId ujId) { + auto now = CLOCK::now(); + _wqsData->addDeadUberJob(qId, ujId, now); +} string ActiveWorker::dump() const { lock_guard lg(_aMtx); @@ -501,8 +422,10 @@ string ActiveWorker::_dump() const { return os.str(); } - -void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, http::CzarContactInfo::Ptr const& czInfo, std::string const& replicationInstanceId, std::string const& replicationAuthKey) { +void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, + http::CzarContactInfo::Ptr const& czInfo, + std::string const& replicationInstanceId, + std::string const& replicationAuthKey) { // Go through wcMap, update existing entries in _awMap, create new entries for those that don't exist, lock_guard awLg(_awMapMtx); for (auto const& [wcKey, wcVal] : wcMap) { @@ -510,39 +433,69 @@ void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, htt if (iter == _awMap.end()) { auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); _awMap[wcKey] = newAW; + if (_czarCancelAfterRestart) { + newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); + } } else { auto aWorker = iter->second; if (!aWorker->compareContactInfo(*wcVal)) { // This should not happen, but try to handle it gracefully if it does. - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " worker contact info changed for " << wcKey << " new=" << wcVal->dump() << " old=" << aWorker->dump()); + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " worker contact info changed for " << wcKey + << " new=" << wcVal->dump() << " old=" << aWorker->dump()); + // If there is existing information, only host and port values will change. aWorker->setWorkerContactInfo(wcVal); } } } } -/* &&& -void ActiveWorkerMap::pruneMap() { - lock_guard awLg(_awMapMtx); - for (auto iter = _awMap.begin(); iter != _awMap.end();) { - auto aWorker = iter->second; - if (aWorker->getWInfo()->timeSinceTouchSeconds() > _maxDeadTimeSeconds) { - iter = _awMap.erase(iter); - } else { - ++iter; - } - } +void ActiveWorkerMap::setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { + _czarCancelAfterRestart = true; + _czarCancelAfterRestartCzId = czId; + _czarCancelAfterRestartQId = lastQId; +} + +ActiveWorker::Ptr ActiveWorkerMap::getActiveWorker(string const& workerId) const { + lock_guard lck(_awMapMtx); + auto iter = _awMap.find(workerId); + if (iter == _awMap.end()) return nullptr; + return iter->second; } -*/ void ActiveWorkerMap::sendActiveWorkersMessages() { // Send messages to each active worker as needed lock_guard lck(_awMapMtx); - for(auto&& [wName, awPtr] : _awMap) { + for (auto&& [wName, awPtr] : _awMap) { awPtr->updateStateAndSendMessages(_timeoutAliveSecs, _timeoutDeadSecs, _maxLifetime); } } +void ActiveWorkerMap::addToDoneDeleteFiles(QueryId qId) { + lock_guard lck(_awMapMtx); + for (auto const& [wName, awPtr] : _awMap) { + awPtr->addToDoneDeleteFiles(qId); + awPtr->removeDeadUberJobsFor(qId); + } +} + +void ActiveWorkerMap::addToDoneKeepFiles(QueryId qId) { + lock_guard lck(_awMapMtx); + for (auto const& [wName, awPtr] : _awMap) { + awPtr->addToDoneKeepFiles(qId); + awPtr->removeDeadUberJobsFor(qId); + } +} + +/* &&& +/// &&& doc +void ActiveWorkerMap::removeDeadUberJobsFor(QueryId qId) { + lock_guard lck(_awMapMtx); + for (auto const& [wName, awPtr] : _awMap) { + awPtr->removeDeadUberJobsFor(qId); + } +} +*/ ActiveWorkerMap::ActiveWorkerMap(std::shared_ptr const& czarConfig) : _timeoutAliveSecs(czarConfig->getActiveWorkerTimeoutAliveSecs()), diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 727ebafeed..fdc55a1e16 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -147,7 +147,6 @@ void Czar::_monitor() { // a separate message (see WorkerCzarComIssue) saying it killed everything that this // czar gave it. Upon getting this message from a worker, this czar will reassign // everything it had sent to that worker. - // TODO:UJ How long should queryId's remain on this list? } } @@ -177,9 +176,6 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->czarId); - // This will block until there is a successful read of the database tables. - _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); - // Tell workers to cancel any queries that were submitted before this restart of Czar. // Figure out which query (if any) was recorded in Czar databases before the restart. // The id will be used as the high-watermark for queries that need to be cancelled. @@ -198,6 +194,9 @@ Czar::Czar(string const& configFilePath, string const& czarName) // This will block until there is a successful read of the database tables. _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + // This will block until there is a successful read of the database tables. + _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + int qPoolSize = _czarConfig->getQdispPoolSize(); int maxPriority = std::max(0, _czarConfig->getQdispMaxPriority()); string vectRunSizesStr = _czarConfig->getQdispVectRunSizes(); diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 7061d7ba09..7c3e74880c 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -342,8 +342,8 @@ bool CzarFamilyMap::_read() { return false; } - // &&& TODO:UJ Before makeNewMaps(), get a list of workers considered to be alive by czar::_activeWorkerMap - // give that list to makeNewMaps, and don't and workers to the maps that aren't on the list.&&& !!! + // &&& TODO:UJ Before makeNewMaps(), get a list of workers considered to be alive by + // czar::_activeWorkerMap // Make the new maps. auto czConfig = cconfig::CzarConfig::instance(); diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index ac65fe5be1..2f74c797e2 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -237,4 +237,17 @@ void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) _activeWorkerMap->sendActiveWorkersMessages(); } +void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) { + lock_guard lck(_mapMtx); + // Add query id to the appropriate list. + if (deleteWorkerResults) { + _activeWorkerMap.addToDoneDeleteFiles(qId); + } else { + _activeWorkerMap.addToDoneKeepFiles(qId); + } + + // With lists updated, send out messages. + _activeWorkerMap.sendActiveWorkersMessages(); +} + } // namespace lsst::qserv::czar diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index 10b0dbcd22..fde3b935c5 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -96,9 +96,19 @@ class CzarRegistry { return _contactMap; } + /// Return _contactMap, the object that the returned pointer points to is + /// constant and no attempts should be made to change it. This + /// function will wait forever for a valid contact map to be ready. + http::WorkerContactInfo::WCMapPtr waitForWorkerContactMap() const; + /// &&& doc void sendActiveWorkersMessages(); + /// Add the query id to the list of queries to end on workers and + /// send the messages, deleting all result files if + /// `deleteWorkerResults` is true. + void endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults); + private: CzarRegistry() = delete; CzarRegistry(std::shared_ptr const& czarConfig, diff --git a/src/http/WorkerQueryStatusData.cc b/src/http/WorkerQueryStatusData.cc index cd254f7c07..e9524a26e1 100644 --- a/src/http/WorkerQueryStatusData.cc +++ b/src/http/WorkerQueryStatusData.cc @@ -46,19 +46,22 @@ namespace lsst::qserv::http { json CzarContactInfo::serializeJson() const { json jsCzar; jsCzar["name"] = czName; - jsCzar["id"]= czId; + jsCzar["id"] = czId; jsCzar["management-port"] = czPort; jsCzar["management-host-name"] = czHostName; + jsCzar["czar-startup-time"] = czStartupTime; return jsCzar; } -CzarContactInfo::Ptr CzarContactInfo::createJson(nlohmann::json const& czJson) { +CzarContactInfo::Ptr CzarContactInfo::createFromJson(nlohmann::json const& czJson) { try { auto czName_ = RequestBodyJSON::required(czJson, "name"); auto czId_ = RequestBodyJSON::required(czJson, "id"); auto czPort_ = RequestBodyJSON::required(czJson, "management-port"); auto czHostName_ = RequestBodyJSON::required(czJson, "management-host-name"); - return create(czName_, czId_, czPort_, czHostName_); + auto czStartupTime_ = RequestBodyJSON::required(czJson, "czar-startup-time"); + return create(czName_, czId_, czPort_, czHostName_, czStartupTime_); + //&&& return create(czName_, czId_, czPort_, czHostName_); } catch (invalid_argument const& exc) { LOGS(_log, LOG_LVL_ERROR, string("CzarContactInfo::createJson invalid ") << exc.what()); } @@ -67,32 +70,52 @@ CzarContactInfo::Ptr CzarContactInfo::createJson(nlohmann::json const& czJson) { std::string CzarContactInfo::dump() const { stringstream os; - os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << czHostName; + //&&& os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << + //czHostName; + os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << czHostName + << " czStartupTime=" << czStartupTime; return os.str(); } - - json WorkerContactInfo::serializeJson() const { + lock_guard lg(_rMtx); + return _serializeJson(); +} + +json WorkerContactInfo::_serializeJson() const { json jsWorker; - jsWorker["id"]= wId; - jsWorker["host"] = wHost; - jsWorker["management-host-name"] = wManagementHost; - jsWorker["management-port"] = wPort; + jsWorker["id"] = wId; + jsWorker["host"] = _wHost; + jsWorker["management-host-name"] = _wManagementHost; + jsWorker["management-port"] = _wPort; + jsWorker["w-startup-time"] = _wStartupTime; return jsWorker; } -WorkerContactInfo::Ptr WorkerContactInfo::createJson(nlohmann::json const& wJson, TIMEPOINT updateTime_) { - LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& a"); +WorkerContactInfo::Ptr WorkerContactInfo::createFromJsonRegistry(string const& wId_, + nlohmann::json const& regJson) { + try { + auto wHost_ = RequestBodyJSON::required(regJson, "host-addr"); + auto wManagementHost_ = RequestBodyJSON::required(regJson, "management-host-name"); + auto wPort_ = RequestBodyJSON::required(regJson, "management-port"); + auto updateTimeInt = RequestBodyJSON::required(regJson, "update-time-ms"); + TIMEPOINT updateTime_ = TIMEPOINT(chrono::milliseconds(updateTimeInt)); + + return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); + } + return nullptr; +} + +WorkerContactInfo::Ptr WorkerContactInfo::createFromJsonWorker(nlohmann::json const& wJson, + TIMEPOINT updateTime_) { try { auto wId_ = RequestBodyJSON::required(wJson, "id"); - LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& b"); auto wHost_ = RequestBodyJSON::required(wJson, "host"); - LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& c"); auto wManagementHost_ = RequestBodyJSON::required(wJson, "management-host-name"); - LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& d"); auto wPort_ = RequestBodyJSON::required(wJson, "management-port"); - LOGS(_log, LOG_LVL_ERROR, "WorkerContactInfo::createJson &&& e"); + return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); } catch (invalid_argument const& exc) { LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); @@ -100,98 +123,92 @@ WorkerContactInfo::Ptr WorkerContactInfo::createJson(nlohmann::json const& wJson return nullptr; } - - string WorkerContactInfo::dump() const { + lock_guard lg(_rMtx); + return _dump(); +} + +string WorkerContactInfo::_dump() const { stringstream os; os << "workerContactInfo{" - << "id=" << wId << " host=" << wHost << " mgHost=" << wManagementHost << " port=" << wPort << "}"; + << "id=" << wId << " host=" << _wHost << " mgHost=" << _wManagementHost << " port=" << _wPort << "}"; return os.str(); } -/* &&& -string ActiveWorker::getStateStr(State st) { - switch (st) { - case ALIVE: return string("ALIVE"); - case QUESTIONABLE: return string("QUESTIONABLE"); - case DEAD: return string("DEAD"); - } - return string("unknown"); -} - - -bool WorkerQueryStatusData::compareContactInfo(WorkerContactInfo const& wcInfo) const { - return _wInfo->isSameContactInfo(wcInfo); -} - -void WorkerQueryStatusData::setWorkerContactInfo(WorkerContactInfo::Ptr const& wcInfo) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " new info=" << wcInfo->dump()); - _wInfo = wcInfo; -} -*/ - - -shared_ptr WorkerQueryStatusData::serializeJson(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime) { - +shared_ptr WorkerQueryStatusData::serializeJson(double maxLifetime) { // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a // message to send to the worker. auto now = CLOCK::now(); - //&&&auto const czarConfig = cconfig::CzarConfig::instance(); - shared_ptr jsWorkerReqPtr = make_shared(); json& jsWorkerR = *jsWorkerReqPtr; jsWorkerR["version"] = http::MetaModule::version; - /* &&& - jsWorkerR["instance_id"] = czarConfig->replicationInstanceId(); - jsWorkerR["auth_key"] = czarConfig->replicationAuthKey(); - */ jsWorkerR["instance_id"] = _replicationInstanceId; jsWorkerR["auth_key"] = _replicationAuthKey; - //&&&jsWorkerR["worker"] = _wInfo->wId; - jsWorkerR["qiddonekeepfiles"] = json::array(); - jsWorkerR["qiddonedeletefiles"] = json::array(); - jsWorkerR["qiddeaduberjobs"] = json::array(); - //&&&jsWorkerR["czar"] = json::object(); jsWorkerR["czar"] = _czInfo->serializeJson(); - //&&&jsWorkerR["worker"] = json::object(); - jsWorkerR["worker"] = _wInfo->serializeJson(); + { + lock_guard lgI(_infoMtx); + if (_wInfo != nullptr) { + jsWorkerR["worker"] = _wInfo->serializeJson(); + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo is null"); + } + } + // Note, old elements in the maps will be deleted after being added to the message + // to keep the czar from keeping track of these forever. + addListsToJson(jsWorkerR, now, maxLifetime); + if (czarCancelAfterRestart) { + jsWorkerR["czarrestart"] = true; + lock_guard mapLg(mapMtx); + jsWorkerR["czarrestartcancelczid"] = czarCancelAfterRestartCzId; + jsWorkerR["czarrestartcancelqid"] = czarCancelAfterRestartQId; + } else { + jsWorkerR["czarrestart"] = false; + } + + return jsWorkerReqPtr; +} +void WorkerQueryStatusData::addListsToJson(json& jsWR, TIMEPOINT tmMark, double maxLifetime) { + jsWR["qiddonekeepfiles"] = json::array(); + jsWR["qiddonedeletefiles"] = json::array(); + jsWR["qiddeaduberjobs"] = json::array(); + lock_guard mapLg(mapMtx); { - auto& jsDoneKeep = jsWorkerR["qiddonekeepfiles"]; - auto iterDoneKeep = _qIdDoneKeepFiles.begin(); - while (iterDoneKeep != _qIdDoneKeepFiles.end()) { + auto& jsDoneKeep = jsWR["qiddonekeepfiles"]; + auto iterDoneKeep = qIdDoneKeepFiles.begin(); + while (iterDoneKeep != qIdDoneKeepFiles.end()) { auto qId = iterDoneKeep->first; jsDoneKeep.push_back(qId); - auto tmStamp = iterDoneKeep->second; - double ageSecs = std::chrono::duration(now - tmStamp).count(); + auto tmTouched = iterDoneKeep->second; + double ageSecs = std::chrono::duration(tmMark - tmTouched).count(); if (ageSecs > maxLifetime) { - iterDoneKeep = _qIdDoneKeepFiles.erase(iterDoneKeep); + iterDoneKeep = qIdDoneKeepFiles.erase(iterDoneKeep); } else { ++iterDoneKeep; } } } { - auto& jsDoneDelete = jsWorkerR["qiddonedeletefiles"]; - auto iterDoneDelete = _qIdDoneDeleteFiles.begin(); - while (iterDoneDelete != _qIdDoneDeleteFiles.end()) { + auto& jsDoneDelete = jsWR["qiddonedeletefiles"]; + auto iterDoneDelete = qIdDoneDeleteFiles.begin(); + while (iterDoneDelete != qIdDoneDeleteFiles.end()) { auto qId = iterDoneDelete->first; jsDoneDelete.push_back(qId); auto tmStamp = iterDoneDelete->second; - double ageSecs = std::chrono::duration(now - tmStamp).count(); + double ageSecs = std::chrono::duration(tmMark - tmStamp).count(); if (ageSecs > maxLifetime) { - iterDoneDelete = _qIdDoneDeleteFiles.erase(iterDoneDelete); + iterDoneDelete = qIdDoneDeleteFiles.erase(iterDoneDelete); } else { ++iterDoneDelete; } } } { - auto& jsDeadUj = jsWorkerR["qiddeaduberjobs"]; - auto iterDeadUjQid = _qIdDeadUberJobs.begin(); - while (iterDeadUjQid != _qIdDeadUberJobs.end()) { - TIMEPOINT oldestTm; // default is zero + auto& jsDeadUj = jsWR["qiddeaduberjobs"]; + auto iterDeadUjQid = qIdDeadUberJobs.begin(); + while (iterDeadUjQid != qIdDeadUberJobs.end()) { + TIMEPOINT youngestTm = TIMEPOINT::max(); // need to find the youngest auto qId = iterDeadUjQid->first; auto& ujIdMap = iterDeadUjQid->second; @@ -200,16 +217,17 @@ shared_ptr WorkerQueryStatusData::serializeJson(double timeoutAliveSecs, d auto iterUjId = ujIdMap.begin(); bool addedUjId = false; + while (iterUjId != ujIdMap.end()) { UberJobId ujId = iterUjId->first; auto tmStamp = iterUjId->second; - if (tmStamp > oldestTm) { - oldestTm = tmStamp; + if (tmStamp < youngestTm) { + youngestTm = tmStamp; } jsUjIds.push_back(ujId); addedUjId = true; - double ageSecs = std::chrono::duration(now - tmStamp).count(); + double ageSecs = std::chrono::duration(tmMark - tmStamp).count(); if (ageSecs > maxLifetime) { iterUjId = ujIdMap.erase(iterUjId); } else { @@ -221,25 +239,20 @@ shared_ptr WorkerQueryStatusData::serializeJson(double timeoutAliveSecs, d jsDeadUj.push_back(jsQidUj); } - if (ujIdMap.empty() - || std::chrono::duration(now - oldestTm).count() > maxLifetime) { - iterDeadUjQid = _qIdDeadUberJobs.erase(iterDeadUjQid); + // If the youngest element was too old, delete the map. + if (ujIdMap.empty() || std::chrono::duration(tmMark - youngestTm).count() > maxLifetime) { + iterDeadUjQid = qIdDeadUberJobs.erase(iterDeadUjQid); } else { ++iterDeadUjQid; } } } - - /* &&& happens in the caller now. - // Start a thread to send the message. (Maybe these should go on the qdisppool? &&&) - // put this in a different function and start the thread.&&&; - _sendStatusMsg(jsWorkerReqPtr); - */ - return jsWorkerReqPtr; } -WorkerQueryStatusData::Ptr WorkerQueryStatusData::createJson(nlohmann::json const& jsWorkerReq, - std::string const& replicationInstanceId, std::string const& replicationAuthKey, TIMEPOINT updateTm) { +WorkerQueryStatusData::Ptr WorkerQueryStatusData::createFromJson(nlohmann::json const& jsWorkerReq, + std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_, + TIMEPOINT updateTm) { LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& a"); try { if (jsWorkerReq["version"] != http::MetaModule::version) { @@ -248,22 +261,26 @@ WorkerQueryStatusData::Ptr WorkerQueryStatusData::createJson(nlohmann::json cons } LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& b"); - auto czInfo_ = CzarContactInfo::createJson(jsWorkerReq["czar"]); + auto czInfo_ = CzarContactInfo::createFromJson(jsWorkerReq["czar"]); LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& c"); - auto wInfo_ = WorkerContactInfo::createJson(jsWorkerReq["worker"], updateTm); + auto wInfo_ = WorkerContactInfo::createFromJsonWorker(jsWorkerReq["worker"], updateTm); LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& d"); if (czInfo_ == nullptr || wInfo_ == nullptr) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson czar or worker info could not be parsed in " << jsWorkerReq); + LOGS(_log, LOG_LVL_ERROR, + "WorkerQueryStatusData::createJson czar or worker info could not be parsed in " + << jsWorkerReq); } - auto wqsData = WorkerQueryStatusData::create(wInfo_, czInfo_, replicationInstanceId, replicationAuthKey); + auto wqsData = + WorkerQueryStatusData::create(wInfo_, czInfo_, replicationInstanceId_, replicationAuthKey_); LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& e"); - - auto parseRes = wqsData->_parseLists(jsWorkerReq, updateTm); - if (!parseRes) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson error reading lists in " << jsWorkerReq); - return nullptr; - } + wqsData->parseLists(jsWorkerReq, updateTm); LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& end"); + bool czarRestart = RequestBodyJSON::required(jsWorkerReq, "czarrestart"); + if (czarRestart) { + auto restartCzarId = RequestBodyJSON::required(jsWorkerReq, "czarrestartcancelczid"); + auto restartQueryId = RequestBodyJSON::required(jsWorkerReq, "czarrestartcancelqid"); + wqsData->setCzarCancelAfterRestart(restartCzarId, restartQueryId); + } return wqsData; } catch (invalid_argument const& exc) { LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); @@ -271,61 +288,150 @@ WorkerQueryStatusData::Ptr WorkerQueryStatusData::createJson(nlohmann::json cons return nullptr; } -bool WorkerQueryStatusData::_parseLists(nlohmann::json const& jsWorkerReq, TIMEPOINT updateTm) { - try { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& a"); - auto& jsQIdDoneKeepFiles = jsWorkerReq["qiddonekeepfiles"]; - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& b"); - for (auto const& qidKeep : jsQIdDoneKeepFiles) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& b1"); - _qIdDoneKeepFiles[qidKeep] = updateTm; - } +void WorkerQueryStatusData::parseLists(nlohmann::json const& jsWR, TIMEPOINT updateTm) { + lock_guard mapLg(mapMtx); + parseListsInto(jsWR, updateTm, qIdDoneKeepFiles, qIdDoneDeleteFiles, qIdDeadUberJobs); +} - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& c"); - auto& jsQIdDoneDeleteFiles = jsWorkerReq["qiddonedeletefiles"]; - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& d"); - for (auto const& qidDelete : jsQIdDoneDeleteFiles) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& d1"); - _qIdDoneDeleteFiles[qidDelete] = updateTm; - } +void WorkerQueryStatusData::parseListsInto(nlohmann::json const& jsWR, TIMEPOINT updateTm, + std::map& doneKeepF, + std::map& doneDeleteF, + std::map>& deadUberJobs) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& a"); + auto& jsQIdDoneKeepFiles = jsWR["qiddonekeepfiles"]; + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& b"); + for (auto const& qidKeep : jsQIdDoneKeepFiles) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& b1"); + doneKeepF[qidKeep] = updateTm; + } - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& e"); - auto& jsQIdDeadUberJobs = jsWorkerReq["qiddeaduberjobs"]; - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f jsQIdDeadUberJobs=" << jsQIdDeadUberJobs); - // Interestingly, !jsQIdDeadUberJobs.empty() doesn't work, but .size() > 0 does. - // Not having the size() check causes issues with the for loop trying to read the - // first element of an empty list, which goes badly. - if (jsQIdDeadUberJobs.size() > 0) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f1"); - for (auto const& qDeadUjs : jsQIdDeadUberJobs) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f1a qDeadUjs=" << qDeadUjs); - QueryId qId = qDeadUjs["qid"]; - auto const& ujIds = qDeadUjs["ujids"]; - auto& mapOfUj = _qIdDeadUberJobs[qId]; - for (auto const& ujId : ujIds) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&& f1d1 qId=" << qId << " ujId=" << ujId); - mapOfUj[ujId] = updateTm; - } + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& c"); + auto& jsQIdDoneDeleteFiles = jsWR["qiddonedeletefiles"]; + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& d"); + for (auto const& qidDelete : jsQIdDoneDeleteFiles) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& d1"); + doneDeleteF[qidDelete] = updateTm; + } + + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& e"); + auto& jsQIdDeadUberJobs = jsWR["qiddeaduberjobs"]; + LOGS(_log, LOG_LVL_ERROR, + "WorkerQueryStatusData::parseListsInto &&& f jsQIdDeadUberJobs=" << jsQIdDeadUberJobs); + // Interestingly, !jsQIdDeadUberJobs.empty() doesn't work, but .size() > 0 does. + // Not having the size() check causes issues with the for loop trying to read the + // first element of an empty list, which goes badly. + if (jsQIdDeadUberJobs.size() > 0) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& f1"); + for (auto const& qDeadUjs : jsQIdDeadUberJobs) { + LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& f1a qDeadUjs=" << qDeadUjs); + QueryId qId = qDeadUjs["qid"]; + auto const& ujIds = qDeadUjs["ujids"]; + auto& mapOfUj = deadUberJobs[qId]; + for (auto const& ujId : ujIds) { + LOGS(_log, LOG_LVL_ERROR, + "WorkerQueryStatusData::parseListsInto &&& f1d1 qId=" << qId << " ujId=" << ujId); + mapOfUj[ujId] = updateTm; } } - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::_parseLists invalid ") << exc.what()); - return false; } - return true; } void WorkerQueryStatusData::addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm) { - auto& ujMap = _qIdDeadUberJobs[qId]; + lock_guard mapLg(mapMtx); + auto& ujMap = qIdDeadUberJobs[qId]; for (auto const ujId : ujIds) { ujMap[ujId] = tm; } } +void WorkerQueryStatusData::addDeadUberJob(QueryId qId, UberJobId ujId, TIMEPOINT tm) { + lock_guard mapLg(mapMtx); + auto& ujMap = qIdDeadUberJobs[qId]; + ujMap[ujId] = tm; +} + +void WorkerQueryStatusData::addToDoneDeleteFiles(QueryId qId) { + lock_guard mapLg(mapMtx); + qIdDoneDeleteFiles[qId] = CLOCK::now(); +} + +void WorkerQueryStatusData::addToDoneKeepFiles(QueryId qId) { + lock_guard mapLg(mapMtx); + qIdDoneKeepFiles[qId] = CLOCK::now(); +} + +void WorkerQueryStatusData::removeDeadUberJobsFor(QueryId qId) { + lock_guard mapLg(mapMtx); + qIdDeadUberJobs.erase(qId); +} + +json WorkerQueryStatusData::serializeResponseJson(uint64_t workerStartupTime) { + // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a + // response. Nothing should be deleted and time is irrelevant for this, so maxLifetime is enormous + // and any time could be used for last contact, but now() is easy. + // This is only called by the worker. As such nothing should be deleted here as the lifetime of + // these elements is determined by the lifetime of the owning UserQueryInfo instance. + double maxLifetime = std::numeric_limits::max(); + auto now = CLOCK::now(); + json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; + jsResp["w-startup-time"] = workerStartupTime; + addListsToJson(jsResp, now, maxLifetime); + return jsResp; +} + +std::pair WorkerQueryStatusData::handleResponseJson(nlohmann::json const& jsResp) { + auto now = CLOCK::now(); + std::map doneKeepF; + std::map doneDeleteF; + std::map> deadUberJobs; + parseListsInto(jsResp, now, doneKeepF, doneDeleteF, deadUberJobs); + + lock_guard mapLg(mapMtx); + // Remove entries from _qIdDoneKeepFiles + for (auto const& [qId, tm] : doneKeepF) { + qIdDoneKeepFiles.erase(qId); + } + + // Remove entries from _qIdDoneDeleteFiles + for (auto const& [qId, tm] : doneDeleteF) { + qIdDoneDeleteFiles.erase(qId); + } + + // Remove entries from _qIdDeadUberJobs + for (auto const& [qId, ujMap] : deadUberJobs) { + auto iter = qIdDeadUberJobs.find(qId); + if (iter != qIdDeadUberJobs.end()) { + auto& deadMap = iter->second; + for (auto const& [ujId, tm] : ujMap) { + deadMap.erase(ujId); + } + if (deadMap.empty()) { + qIdDeadUberJobs.erase(iter); + } + } + } + + bool workerRestarted = false; + auto workerStartupTime = RequestBodyJSON::required(jsResp, "w-startup-time"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " workerStartupTime=" << workerStartupTime); + if (!_wInfo->checkWStartupTime(workerStartupTime)) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " startup time for worker=" << _wInfo->dump() + << " changed to=" << workerStartupTime << " Assuming worker restarted"); + workerRestarted = true; + } + return {true, workerRestarted}; +} + string WorkerQueryStatusData::dump() const { + lock_guard lgI(_infoMtx); + return _dump(); +} + +string WorkerQueryStatusData::_dump() const { stringstream os; os << "ActiveWorker " << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); return os.str(); } -} // namespace lsst::qserv::czar +} // namespace lsst::qserv::http diff --git a/src/http/WorkerQueryStatusData.h b/src/http/WorkerQueryStatusData.h index f0f6c1aaa1..c56c148b0d 100644 --- a/src/http/WorkerQueryStatusData.h +++ b/src/http/WorkerQueryStatusData.h @@ -35,17 +35,14 @@ #include "global/clock_defs.h" #include "global/intTypes.h" - // This header declarations namespace lsst::qserv::http { -/// &&& doc +/// This class just contains the czar id and network contact information. class CzarContactInfo { public: using Ptr = std::shared_ptr; - std::string cName(const char* fnc) const { - return std::string("CzarContactInfo") + fnc; - } + std::string cName(const char* fnc) const { return std::string("CzarContactInfo") + fnc; } CzarContactInfo() = delete; CzarContactInfo(CzarContactInfo const&) = default; @@ -53,43 +50,39 @@ class CzarContactInfo { /// &&& doc bool compare(CzarContactInfo const& other) { - return (czName == other.czName && czId == other.czId && czPort == other.czPort && czHostName == other.czHostName); + return (czName == other.czName && czId == other.czId && czPort == other.czPort && + czHostName == other.czHostName); } - static Ptr create(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_) { - return Ptr(new CzarContactInfo(czName_, czId_, czPort_, czHostName_)); + static Ptr create(std::string const& czName_, CzarIdType czId_, int czPort_, + std::string const& czHostName_, uint64_t czStartupTime_) { + return Ptr(new CzarContactInfo(czName_, czId_, czPort_, czHostName_, czStartupTime_)); } - static Ptr createJson(nlohmann::json const& czarJson); + static Ptr createFromJson(nlohmann::json const& czarJson); - std::string const czName; ///< czar "name" - CzarIdType const czId; ///< czar "id" - int const czPort; ///< czar "management-port" - std::string const czHostName; ///< czar "management-host-name" + std::string const czName; ///< czar "name" + CzarIdType const czId; ///< czar "id" + int const czPort; ///< czar "management-port" + std::string const czHostName; ///< czar "management-host-name" + uint64_t const czStartupTime; ///< czar startup time /// &&& doc nlohmann::json serializeJson() const; - /// &&& doc - //&&&bool parse(nlohmann::json const& czarJson); - std::string dump() const; - /* &&& - auto& jsWCzar = jsWorkerR["czar"]; - jsWCzar["name"] = czarConfig->name(); - jsWCzar["id"]= czarConfig->id(); - jsWCzar["management-port"] = czarConfig->replicationHttpPort(); - jsWCzar["management-host-name"] = util::get_current_host_fqdn(); - */ + private: - CzarContactInfo(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_) - : czName(czName_), czId(czId_), czPort(czPort_), czHostName(czHostName_) {} + CzarContactInfo(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_, + uint64_t czStartupTime_) + : czName(czName_), + czId(czId_), + czPort(czPort_), + czHostName(czHostName_), + czStartupTime(czStartupTime_) {} }; - -/// &&& doc This class just contains the worker id and network communication -/// information, but it may be desirable to store connections to the -/// worker here as well. +/// This class just contains the worker id and network communication information. class WorkerContactInfo { public: using Ptr = std::shared_ptr; @@ -97,39 +90,62 @@ class WorkerContactInfo { using WCMap = std::unordered_map; using WCMapPtr = std::shared_ptr; - static Ptr create(std::string const& wId_, std::string const& wHost_, - std::string const& wManagementHost_, int wPort_, TIMEPOINT updateTime_) { - return Ptr(new WorkerContactInfo(wId_, wHost_, - wManagementHost_, wPort_, updateTime_)); + static Ptr create(std::string const& wId_, std::string const& wHost_, std::string const& wManagementHost_, + int wPort_, TIMEPOINT updateTime_) { + return Ptr(new WorkerContactInfo(wId_, wHost_, wManagementHost_, wPort_, updateTime_)); } - /// &&& doc - static Ptr createJson(nlohmann::json const& workerJson, TIMEPOINT updateTime); + /// &&& doc Used to create WorkerQueryStatusData object from a registry json message. + static Ptr createFromJsonRegistry(std::string const& wId_, nlohmann::json const& regJson); + + /// &&& doc Used to create WorkerQueryStatusData object from a worker json message. + static Ptr createFromJsonWorker(nlohmann::json const& workerJson, TIMEPOINT updateTime); /// &&& doc nlohmann::json serializeJson() const; std::string cName(const char* fn) { return std::string("WorkerContactInfo::") + fn; } - /// &&& make private - WorkerContactInfo(std::string const& wId_, std::string const& wHost_, - std::string const& wManagementHost_, int wPort_, TIMEPOINT updateTime_) - : wId(wId_), - wHost(wHost_), - wManagementHost(wManagementHost_), - wPort(wPort_) { - regUpdateTime(updateTime_); + std::string const wId; ///< key, this is the one thing that cannot change. + + std::string getWHost() const { + std::lock_guard lg(_rMtx); + return _wHost; + } + + std::string getWManagementHost() const { + std::lock_guard lg(_rMtx); + return _wManagementHost; + } + + int getWPort() const { + std::lock_guard lg(_rMtx); + return _wPort; } - std::string const wId; ///< key - std::string const wHost; ///< "host-addr" entry. - std::string const wManagementHost; ///< "management-host-name" entry. - int const wPort; ///< "management-port" entry. + /// &&doc + void changeBaseInfo(WorkerContactInfo const& other) { + auto [oWId, oWHost, oWManagementHost, oWPort] = other.getAll(); + std::lock_guard lg(_rMtx); + _wHost = oWHost; + _wManagementHost = oWManagementHost; + _wPort = oWPort; + } - /// Return true if all members, aside from updateTime, are equal. + /// @return wId - workerId + /// @return _wHost - worker host + /// @return _wManagementHost - management host + /// @return _wPort - worker port + std::tuple getAll() const { + std::lock_guard lg(_rMtx); + return {wId, _wHost, _wManagementHost, _wPort}; + } + + /// Return true if communication related items are the same. bool isSameContactInfo(WorkerContactInfo const& other) const { - return (wId == other.wId && wHost == other.wHost && wManagementHost == other.wManagementHost && - wPort == other.wPort); + auto [oWId, oWHost, oWManagementHost, oWPort] = other.getAll(); + std::lock_guard lg(_rMtx); + return (wId == oWId && _wHost == oWHost && _wManagementHost == oWManagementHost && _wPort == oWPort); } void regUpdateTime(TIMEPOINT updateTime) { @@ -148,85 +164,206 @@ class WorkerContactInfo { return _regUpdate; } + /* &&& + /// Sets _wStartupTime to startupTime, but only if _wStartupTime was 0. + /// @returns true if _wStartupTime was set. + bool setWStartupTime(uint64_t startupTime) { //&&& del if not used + std::lock_guard lg(_rMtx); + if (_wStartupTime == 0) { + _wStartupTime = startupTime; + return true; + } + return false; + } + */ + + /// @return true if startupTime equals _wStartupTime or _wStartupTime was never set, + /// if _wStartupTime was never set, it is set to startupTime. + /// @return false indicates the worker was restarted and all associated jobs need + /// re-assignment. + bool checkWStartupTime(uint64_t startupTime) { + std::lock_guard lg(_rMtx); + if (_wStartupTime == startupTime) { + return true; + } + if (_wStartupTime == 0) { + _wStartupTime = startupTime; + return true; + } + _wStartupTime = startupTime; + return false; + } + + uint64_t getWStartupTime() const { + std::lock_guard lg(_rMtx); + return _wStartupTime; + } + std::string dump() const; private: + WorkerContactInfo(std::string const& wId_, std::string const& wHost_, std::string const& wManagementHost_, + int wPort_, TIMEPOINT updateTime_) + : wId(wId_), _wHost(wHost_), _wManagementHost(wManagementHost_), _wPort(wPort_) { + regUpdateTime(updateTime_); + } + + // _rMtx must be locked before calling + std::string _dump() const; + + // _rMtx must be locked before calling + nlohmann::json _serializeJson() const; + + std::string _wHost; ///< "host-addr" entry. + std::string _wManagementHost; ///< "management-host-name" entry. + int _wPort; ///< "management-port" entry. + /// Last time the registry heard from this worker. The ActiveWorker class /// will use this to determine the worker's state. /// &&& Store in seconds since epoch to make atomic? TIMEPOINT _regUpdate; - mutable std::mutex _rMtx; ///< protects _regUpdate -}; + /// "w-startup-time", it's value is set to zero until the real value is + /// received from the worker. Once it is non-zero, any change indicates + /// the worker was restarted and all UberJobs that were assigned there + /// need to be unassigned. On the worker, this should always be set from + /// foreman()->getStartupTime(); + uint64_t _wStartupTime = 0; + mutable std::mutex _rMtx; ///< protects _regUpdate +}; -/// &&& doc -class WorkerQueryStatusData { +/// This classes purpose is to be a structure to store and transfer information +/// about which queries have been completed or cancelled on the worker. This +/// class contains the functions that encode and decode the data they contain +/// to and from a json format. +class WorkerQueryStatusData { public: using Ptr = std::shared_ptr; - /* &&& - enum State { - ALIVE = 0, - QUESTIONABLE, - DEAD - }; - */ - WorkerQueryStatusData() = delete; WorkerQueryStatusData(WorkerQueryStatusData const&) = delete; WorkerQueryStatusData& operator=(WorkerQueryStatusData const&) = delete; - std::string cName(const char* fName) { - return std::string("WorkerQueryStatusData::") + fName + " " + ((_wInfo == nullptr) ? "?" : _wInfo->wId); + std::string cName(const char* fName) { return std::string("WorkerQueryStatusData::") + fName; } + + static Ptr create(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_, + std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { + return Ptr(new WorkerQueryStatusData(wInfo_, czInfo_, replicationInstanceId_, replicationAuthKey_)); } - //&&&static std::string getStateStr(State st); + /// &&& doc Used to create WorkerQueryStatusData object from a worker json message. + static Ptr createFromJson(nlohmann::json const& czarJson, std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_, TIMEPOINT updateTm); + + ~WorkerQueryStatusData() = default; - static Ptr create(WorkerContactInfo::Ptr const& wInfo, CzarContactInfo::Ptr const& czInfo, - std::string const& replicationInstanceId, std::string const& replicationAuthKey) { - return Ptr(new WorkerQueryStatusData(wInfo, czInfo, replicationInstanceId, replicationAuthKey)); + void setWInfo(WorkerContactInfo::Ptr const& wInfo_) { + std::lock_guard lgI(_infoMtx); + if (_wInfo == nullptr) { + _wInfo = wInfo_; + return; + } + if (wInfo_ != nullptr) { + // This only change host and port values of _wInfo. + _wInfo->changeBaseInfo(*wInfo_); + } } - /// &&& doc - static Ptr createJson(nlohmann::json const& czarJson, - std::string const& replicationInstanceId, std::string const& replicationAuthKey, TIMEPOINT updateTm); + WorkerContactInfo::Ptr getWInfo() const { + std::lock_guard lgI(_infoMtx); + return _wInfo; + } + CzarContactInfo::Ptr getCzInfo() const { return _czInfo; } + /// doc &&& + void addDeadUberJob(QueryId qId, UberJobId ujId, TIMEPOINT tm); - ~WorkerQueryStatusData() = default; + /// &&& doc + void addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm); - WorkerContactInfo::Ptr getWInfo() const { return _wInfo; } + /// &&& doc + void addToDoneDeleteFiles(QueryId qId); /// &&& doc - void addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm); + void addToDoneKeepFiles(QueryId qId); - std::string dump() const; + /// &&& doc + void removeDeadUberJobsFor(QueryId qId); -//&&&private: - WorkerQueryStatusData(WorkerContactInfo::Ptr const& wInfo, CzarContactInfo::Ptr const& czInfo, - std::string const& replicationInstanceId, std::string const& replicationAuthKey) - : _wInfo(wInfo), _czInfo(czInfo), - _replicationInstanceId(replicationInstanceId), _replicationAuthKey(replicationAuthKey) {} + void setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { + std::lock_guard mapLg(mapMtx); + czarCancelAfterRestart = true; + czarCancelAfterRestartCzId = czId; + czarCancelAfterRestartQId = lastQId; + } - std::map _qIdDoneKeepFiles; ///< &&& doc - limit reached - std::map _qIdDoneDeleteFiles; ///< &&& doc -cancelled/finished - std::map> _qIdDeadUberJobs; ///< &&& doc + bool isCzarRestart() const { return czarCancelAfterRestart; } + CzarIdType getCzarRestartCzarId() const { return czarCancelAfterRestartCzId; } + QueryId getCzarRestartQueryId() const { return czarCancelAfterRestartQId; } - /// &&& TODO:UJ Worth the effort to inform worker of killed UberJobs? - //std::map> _killedUberJobs; + /// Create a json object held by a shared pointer to use as a message. + /// Old objects in this instance will be removed after being added to the + /// json message. + std::shared_ptr serializeJson(double maxLifetime); - WorkerContactInfo::Ptr _wInfo; ///< &&& doc - CzarContactInfo::Ptr _czInfo; //< &&& doc + /// Add contents of qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs to `jsWR`, + /// and remove map elements that have an age (tmMark - element.touchTime) greater + /// than maxLifetime. + void addListsToJson(nlohmann::json& jsWR, TIMEPOINT tmMark, double maxLifetime); - std::string const _replicationInstanceId; ///< &&& doc - std::string const _replicationAuthKey; ///< &&& doc + /// &&& doc + /// @throws std::invalid_argument + void parseLists(nlohmann::json const& jsWR, TIMEPOINT updateTm); /// &&& doc - std::shared_ptr serializeJson(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime); + //&&&nlohmann::json serializeResponseJson(); + nlohmann::json serializeResponseJson(uint64_t workerStartupTime); /// &&& doc - /// @throws std::invalid_argument - bool _parseLists(nlohmann::json const& jsWorkerReq, TIMEPOINT updateTm); // &&& delete after basic testing + //&&&bool handleResponseJson(nlohmann::json const& jsResp); + std::pair handleResponseJson(nlohmann::json const& jsResp); + + /// &&& doc + ///&&&void handleCzarRestart(); + + /// &&& doc + static void parseListsInto(nlohmann::json const& jsWR, TIMEPOINT updateTm, + std::map& doneKeepF, + std::map& doneDeleteF, + std::map>& deadUberJobs); + + std::string dump() const; + + // Making these private requires member functions to be written + // that cause issues with linking. All of the workarounds are ugly. + std::map qIdDoneKeepFiles; ///< &&& doc - limit reached + std::map qIdDoneDeleteFiles; ///< &&& doc -cancelled/finished + std::map> qIdDeadUberJobs; ///< &&& doc + std::atomic czarCancelAfterRestart = false; + CzarIdType czarCancelAfterRestartCzId = 0; + QueryId czarCancelAfterRestartQId = 0; + /// Protects _qIdDoneKeepFiles, _qIdDoneDeleteFiles, _qIdDeadUberJobs, + /// and czarCancelAfter variables. + mutable std::mutex mapMtx; + +private: + WorkerQueryStatusData(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_, + std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) + : _wInfo(wInfo_), + _czInfo(czInfo_), + _replicationInstanceId(replicationInstanceId_), + _replicationAuthKey(replicationAuthKey_) {} + + WorkerContactInfo::Ptr _wInfo; ///< &&& doc + CzarContactInfo::Ptr const _czInfo; //< &&& doc + mutable std::mutex _infoMtx; ///< protects wInfo + + std::string const _replicationInstanceId; ///< &&& doc + std::string const _replicationAuthKey; ///< &&& doc + + /// _infoMtx must be locked before calling. + std::string _dump() const; }; } // namespace lsst::qserv::http diff --git a/src/http/testStatusData.cc b/src/http/testStatusData.cc index 97767dd9fa..d9f5377116 100644 --- a/src/http/testStatusData.cc +++ b/src/http/testStatusData.cc @@ -44,97 +44,126 @@ using namespace lsst::qserv::http; BOOST_AUTO_TEST_SUITE(Suite) BOOST_AUTO_TEST_CASE(CzarContactInfo) { - string const replicationInstanceId = "repliInstId"; string const replicationAuthKey = "repliIAuthKey"; - string const cName("czar_name"); - lsst::qserv::CzarIdType const cId = 32; - int cPort = 2022; - string const cHost("cz_host"); + uint64_t cxrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 5s); + uint64_t wkrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 10s); + + string const czrName("czar_name"); + lsst::qserv::CzarIdType const czrId = 32; + int czrPort = 2022; + string const czrHost("cz_host"); - auto czarA = lsst::qserv::http::CzarContactInfo::create(cName, cId, cPort, cHost); + //&&&auto czarA = lsst::qserv::http::CzarContactInfo::create(czrName, czrId, czrPort, czrHost); + auto czarA = lsst::qserv::http::CzarContactInfo::create(czrName, czrId, czrPort, czrHost, cxrStartTime); LOGS_ERROR("&&& a czarA=" << czarA->dump()); auto czarAJs = czarA->serializeJson(); LOGS_ERROR("&&& b czarAJs=" << czarAJs); - auto czarB = lsst::qserv::http::CzarContactInfo::createJson(czarAJs); + auto czarB = lsst::qserv::http::CzarContactInfo::createFromJson(czarAJs); LOGS_ERROR("&&& c czarB=" << czarB); BOOST_REQUIRE(czarA->compare(*czarB)); - auto czarC = lsst::qserv::http::CzarContactInfo::create("different", cId, cPort, cHost); + //&&&auto czarC = lsst::qserv::http::CzarContactInfo::create("different", czrId, czrPort, czrHost); + auto czarC = + lsst::qserv::http::CzarContactInfo::create("different", czrId, czrPort, czrHost, cxrStartTime); BOOST_REQUIRE(!czarA->compare(*czarC)); auto start = lsst::qserv::CLOCK::now(); auto workerA = WorkerContactInfo::create("sd_workerA", "host_w1", "mgmhost_a", 3421, start); + auto workerB = WorkerContactInfo::create("sd_workerB", "host_w2", "mgmhost_a", 3421, start); auto workerC = WorkerContactInfo::create("sd_workerC", "host_w3", "mgmhost_b", 3422, start); + LOGS_ERROR("&&& d workerA=" << workerA->dump()); auto jsWorkerA = workerA->serializeJson(); LOGS_ERROR("&&& e jsWorkerA=" << jsWorkerA); auto start1Sec = start + 1s; - auto workerA1 = WorkerContactInfo::createJson(jsWorkerA, start1Sec); + auto workerA1 = WorkerContactInfo::createFromJsonWorker(jsWorkerA, start1Sec); LOGS_ERROR("&&& f workerA1=" << workerA1->dump()); BOOST_REQUIRE(workerA->isSameContactInfo(*workerA1)); // WorkerQueryStatusData - auto wqsdA = lsst::qserv::http::WorkerQueryStatusData::create(workerA, czarA, replicationInstanceId, replicationAuthKey); + auto wqsdA = lsst::qserv::http::WorkerQueryStatusData::create(workerA, czarA, replicationInstanceId, + replicationAuthKey); LOGS_ERROR("&&& g wqsdA=" << wqsdA->dump()); - double timeoutAliveSecs = 100.0; - double timeoutDeadSecs = 2*timeoutAliveSecs; + //&&&double timeoutAliveSecs = 100.0; + //&&&double timeoutDeadSecs = 2*timeoutAliveSecs; double maxLifetime = 300.0; - auto jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + auto jsDataA = wqsdA->serializeJson(maxLifetime); LOGS_ERROR("&&& h jsDataA=" << *jsDataA); // Check that empty lists work. - auto wqsdA1 = lsst::qserv::http::WorkerQueryStatusData::createJson(*jsDataA, replicationInstanceId, replicationAuthKey, start1Sec); + auto wqsdA1 = lsst::qserv::http::WorkerQueryStatusData::createFromJson(*jsDataA, replicationInstanceId, + replicationAuthKey, start1Sec); LOGS_ERROR("&&& i wqsdA1=" << wqsdA1->dump()); - auto jsDataA1 = wqsdA1->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + LOGS_ERROR("&&& i wqsdA=" << wqsdA->dump()); + auto jsDataA1 = wqsdA1->serializeJson(maxLifetime); + LOGS_ERROR("&&& i jsDataA1=" << *jsDataA1); + LOGS_ERROR("&&& i jsDataA=" << *jsDataA); BOOST_REQUIRE(*jsDataA == *jsDataA1); - - vector qIdsDelFiles = { 7, 8, 9, 15, 25, 26, 27, 30 }; - vector qIdsKeepFiles = { 1, 2, 3, 4, 6, 10, 13, 19, 33 }; + vector qIdsDelFiles = {7, 8, 9, 15, 25, 26, 27, 30}; + vector qIdsKeepFiles = {1, 2, 3, 4, 6, 10, 13, 19, 33}; for (auto const qIdDF : qIdsDelFiles) { - wqsdA->_qIdDoneDeleteFiles[qIdDF] = start; + wqsdA->qIdDoneDeleteFiles[qIdDF] = start; } - jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + jsDataA = wqsdA->serializeJson(maxLifetime); LOGS_ERROR("&&& j jsDataA=" << jsDataA); BOOST_REQUIRE(*jsDataA != *jsDataA1); for (auto const qIdKF : qIdsKeepFiles) { - wqsdA->_qIdDoneKeepFiles[qIdKF] = start; + wqsdA->qIdDoneKeepFiles[qIdKF] = start; } wqsdA->addDeadUberJobs(12, {1, 3}, start); LOGS_ERROR("&&& i wqsdA=" << wqsdA->dump()); - jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + jsDataA = wqsdA->serializeJson(maxLifetime); LOGS_ERROR("&&& j jsDataA=" << *jsDataA); auto start5Sec = start + 5s; - auto workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createJson(*jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); - auto jsWorkerAFromJson = workerAFromJson->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + auto workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createFromJson( + *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); + auto jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); wqsdA->addDeadUberJobs(12, {34}, start5Sec); wqsdA->addDeadUberJobs(91, {77}, start5Sec); wqsdA->addDeadUberJobs(1059, {1, 4, 6, 7, 8, 10, 3, 22, 93}, start5Sec); - jsDataA = wqsdA->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + jsDataA = wqsdA->serializeJson(maxLifetime); LOGS_ERROR("&&& k jsDataA=" << *jsDataA); BOOST_REQUIRE(*jsDataA != *jsWorkerAFromJson); - workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createJson(*jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); - jsWorkerAFromJson = workerAFromJson->serializeJson(timeoutAliveSecs, timeoutDeadSecs, maxLifetime); + workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createFromJson( + *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); + jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); LOGS_ERROR("&&& l jsWorkerAFromJson=" << *jsWorkerAFromJson); BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); + // Make the response, which contains lists of the items handled by the workers. + auto jsWorkerResp = workerAFromJson->serializeResponseJson(wkrStartTime); + + // test removal of elements after response. + BOOST_REQUIRE(!wqsdA->qIdDoneDeleteFiles.empty()); + BOOST_REQUIRE(!wqsdA->qIdDoneKeepFiles.empty()); + BOOST_REQUIRE(!wqsdA->qIdDeadUberJobs.empty()); + + wqsdA->handleResponseJson(jsWorkerResp); + auto [respSuccess, workerRestarted] = wqsdA->handleResponseJson(jsWorkerResp); + BOOST_REQUIRE(respSuccess == true); + BOOST_REQUIRE(workerRestarted == false); + + BOOST_REQUIRE(wqsdA->qIdDoneDeleteFiles.empty()); + BOOST_REQUIRE(wqsdA->qIdDoneKeepFiles.empty()); + BOOST_REQUIRE(wqsdA->qIdDeadUberJobs.empty()); } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/proto/worker.proto b/src/proto/worker.proto index 9bb29fb03d..1ccefaf5f9 100644 --- a/src/proto/worker.proto +++ b/src/proto/worker.proto @@ -66,4 +66,3 @@ message ResponseData { required uint32 rowcount = 2; required uint64 transmitsize = 3; } - diff --git a/src/protojson/ScanTableInfo.h b/src/protojson/ScanTableInfo.h index 8cf6811fd3..6f31d9f676 100644 --- a/src/protojson/ScanTableInfo.h +++ b/src/protojson/ScanTableInfo.h @@ -36,7 +36,7 @@ namespace lsst::qserv::protojson { /// Structure to store shared scan information for a single table. /// -struct ScanTableInfo { // &&& check if still useful +struct ScanTableInfo { // &&& check if still useful using ListOf = std::vector; ScanTableInfo() = default; diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 15e8b26b66..c2daea541e 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -280,7 +280,7 @@ void Executive::addAndQueueUberJob(shared_ptr const& uj) { } } -void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { +void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { // &&& put file collect in the pool ??? if (_scanInteractive) { _qdispPool->queCmd(cmd, 3); } else { @@ -289,20 +289,15 @@ void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { } void Executive::runUberJob(std::shared_ptr const& uberJob) { - /// TODO:UJ delete useqdisppool, only set to false if problems during testing - bool const useqdisppool = true; - if (useqdisppool) { - auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; - - auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); - _jobStartCmdList.push_back(cmd); - if (_scanInteractive) { - _qdispPool->queCmd(cmd, 0); - } else { - _qdispPool->queCmd(cmd, 1); - } + + auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; + + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(runUberJobFunc)); + _jobStartCmdList.push_back(cmd); + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 0); } else { - uberJob->runUberJob(); + _qdispPool->queCmd(cmd, 1); } } @@ -387,6 +382,27 @@ void Executive::addUberJobs(std::vector> const& uJobsTo } } +void Executive::killIncompleteUberJobsOn(std::string const& restartedWorkerId) { + // Work with a copy to reduce lock time. + std::map> ujobsMap; + { + lock_guard lck(_uberJobsMapMtx); + ujobsMap = _uberJobsMap; + } + for (auto&& [ujKey, uj] : ujobsMap) { + if (uj == nullptr) continue; + auto wContactInfo = uj->getWorkerContactInfo(); + if (wContactInfo->wId == restartedWorkerId) { + if (uj->getStatus()->getState() != qmeta::JobStatus::COMPLETE) { + // All jobs in the uberjob will be set as unassigned, which + // will lead to Czar::_monitor() reassigning them to new + // UberJobs. (Unless this query was cancelled.) + uj->killUberJob(); + } + } + } +} + string Executive::dumpUberJobCounts() const { stringstream os; os << "exec=" << getIdStr(); @@ -635,14 +651,10 @@ void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { } } -void Executive::sendWorkerCancelMsg(bool deleteResults) { - // TODO:UJ need to send a message to the worker that the query is cancelled and all result files - // should be delete - LOGS(_log, LOG_LVL_ERROR, - "TODO:UJ NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId " - "+ " - "queryId. " +void Executive::sendWorkersEndMsg(bool deleteResults) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " terminating this query deleteResults=" << deleteResults); + czar::Czar::getCzar()->getCzarRegistry()->endUserQueryOnWorkers(_id, deleteResults); } int Executive::getNumInflight() const { diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index bb50645d81..732c0d70d3 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -140,7 +140,7 @@ class Executive : public std::enable_shared_from_this { void queueFileCollect(std::shared_ptr const& cmd); /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. - void queueFileCollect(PriorityCommand::Ptr const& cmd); + void queueFileCollect(std::shared_ptr const& cmd); // &&& delete ??? /// Waits for all jobs on _jobStartCmdList to start. This should not be called /// before ALL jobs have been added to the pool. diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h index 1a42394578..b6b18d3252 100644 --- a/src/qdisp/JobBase.h +++ b/src/qdisp/JobBase.h @@ -58,8 +58,9 @@ class JobBase : public std::enable_shared_from_this { virtual QueryId getQueryId() const = 0; virtual UberJobId getJobId() const = 0; virtual std::string const& getIdStr() const = 0; - virtual std::shared_ptr getQdispPool() = 0; - //&&& virtual std::string const& getPayload() const = 0; ///< const& in return type is essential for xrootd + //&&&virtual std::shared_ptr getQdispPool() = 0; + //&&& virtual std::string const& getPayload() const = 0; ///< const& in return type is essential for + // xrootd virtual std::shared_ptr getRespHandler() = 0; virtual std::shared_ptr getStatus() = 0; virtual bool getScanInteractive() const = 0; diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 0ddc24549f..21b8dd8174 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -53,14 +53,14 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j _jobStatus(jobStatus), _qid(qid), _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { - _qdispPool = executive->getQdispPool(); + //&&&_qdispPool = executive->getQdispPool(); LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_TRACE, "~JobQuery QID=" << _idStr); } /// Cancel response handling. Return true if this is the first time cancel has been called. -bool JobQuery::cancel(bool superfluous) { /// &&& This can probably be simplified more +bool JobQuery::cancel(bool superfluous) { /// &&& This can probably be simplified more QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); if (_cancelled.exchange(true) == false) { LOGS(_log, LOG_LVL_TRACE, "JobQuery::cancel() " << superfluous); diff --git a/src/qdisp/SharedResources.h b/src/qdisp/SharedResources.h new file mode 100644 index 0000000000..6ca6eb8a31 --- /dev/null +++ b/src/qdisp/SharedResources.h @@ -0,0 +1,64 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +#ifndef LSST_QSERV_SHAREDRESOURCES_H +#define LSST_QSERV_SHAREDRESOURCES_H + +// System headers +#include + +namespace lsst::qserv::util { // &&& delete +class QdispPool; +} + +namespace lsst::qserv::qdisp { + +/// Put resources that all Executives need to share in one class to reduce +/// the number of arguments passed. +/// This class should be kept simple so it can easily be included in headers +/// without undue compiler performances problems. + // &&& there's nothing in here but qdisppool!? Try to delete, but there + // &&& will probably be unit test issues. +class SharedResources { +public: + using Ptr = std::shared_ptr; + + static Ptr create(std::shared_ptr const& qdispPool) { + return Ptr(new SharedResources(qdispPool)); + } + + SharedResources() = delete; + SharedResources(SharedResources const&) = delete; + SharedResources& operator=(SharedResources const&) = delete; + ~SharedResources() = default; + + std::shared_ptr getQdispPool() { return _qdispPool; } //&&& delete + +private: + SharedResources(std::shared_ptr const& qdispPool) : _qdispPool(qdispPool) {} + + /// Thread pool for handling Responses from XrdSsi. + std::shared_ptr _qdispPool; +}; + +} // namespace lsst::qserv::qdisp + +#endif // LSST_QSERV_SHAREDRESOURCES_H diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 61d7c31f03..bd8ca95b71 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -236,6 +236,15 @@ bool UberJob::isQueryCancelled() { return exec->getCancelled(); } +bool UberJob::getScanInteractive() const { + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " _executive == nullptr"); + return false; // Safer to assume the worst. + } + return exec->getScanInteractive(); +} + bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg) { // must be locked _jobsMtx auto currentState = _jobStatus->getState(); @@ -596,7 +605,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ ujPtr->_importResultFinish(resultRows); }; - auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); + auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); exec->queueFileCollect(cmd); // If the query meets the limit row complete complete criteria, it will start @@ -726,6 +735,37 @@ nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& e return jsRet; } +void UberJob::killUberJob() { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " stopping this UberJob and re-assigning jobs."); + + auto exec = _executive.lock(); + if (exec == nullptr || isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); + return; + } + + if (exec->isLimitRowComplete()) { + int dataIgnored = exec->incrDataIgnoredCount(); + if ((dataIgnored - 1) % 1000 == 0) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already."); + } + return; + } + + // Put this UberJob on the list of UberJobs that the worker should drop. + auto activeWorkerMap = czar::Czar::getCzar()->getActiveWorkerMap(); + auto activeWorker = activeWorkerMap->getActiveWorker(_wContactInfo->wId); + if (activeWorker != nullptr) { + activeWorker->addDeadUberJob(_queryId, _uberJobId); + } + + _unassignJobs(); + // Let Czar::_monitor reassign jobs - other UberJobs are probably being killed + // so waiting probably gets a better distribution. If this is deemed to slow, + // then exec->assignJobsToUberJobs() could be called here. + return; +} + std::ostream& UberJob::dumpOS(std::ostream& os) const { os << "(jobs sz=" << _jobs.size() << "("; lock_guard lockJobsMtx(_jobsMtx); diff --git a/src/util/xrootd.cc b/src/util/xrootd.cc new file mode 100644 index 0000000000..bde271719a --- /dev/null +++ b/src/util/xrootd.cc @@ -0,0 +1,91 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2009-2015 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +// xrootd.h -- Helper funcitons for xrootd-based dispatch + +#include "util/xrootd.h" + +// System headers +#include +#include + +// Third-party headers +#include "boost/format.hpp" + +/// &&& file seems unused, delete if possible + +namespace lsst::qserv::util { + +std::string makeUrl(char const* hostport, char const* typeStr, int chunk) { + std::stringstream s; + s << chunk; + // boost::format version is 5x slower. + // std::string s = (boost::format("%d") % chunk).str(); + return makeUrl(hostport, typeStr, s.str()); +} + +std::string makeUrl(char const* hostport, std::string const& path) { + return makeUrl(hostport, nullptr, path); +} + +std::string makeUrl(char const* hostport, char const* typeStr, std::string const& s, char mode) { + // typeStr is either "query" or "result" + if (!hostport) { + hostport = ::getenv("QSERV_XRD"); + if (!hostport) { + // use local host name if nothing is specified + hostport = "localhost:1094"; + } + } +#if 0 + char* user = "qsmaster"; + boost::format f("xroot://%s@%s//%s/%s"); + return (f % user % hostport % typeStr % s).str(); +#else + // This is ~8.5x faster than the boost::format version. + std::string pfx = "xroot://"; + std::string user("qsmaster"); + std::string tstr; + std::string ret; + if (typeStr) tstr = typeStr; + + if (mode != '\0') { + user += "."; + user += mode; + } + ret.reserve(pfx.size() + user.size() + 1 + 2 + 1 + tstr.size() + s.size()); + ret += pfx; + ret += user; + ret += "@"; + ret += hostport; + ret += "/"; + if (typeStr) { + ret += "/"; + ret += typeStr; + ret += "/"; + } // else: assume s contains leading "/" + ret += s; + return ret; +#endif +} + +} // namespace lsst::qserv::util diff --git a/src/util/xrootd.h b/src/util/xrootd.h new file mode 100644 index 0000000000..947db582bd --- /dev/null +++ b/src/util/xrootd.h @@ -0,0 +1,44 @@ +// -*- LSST-C++ -*- + +/* + * LSST Data Management System + * Copyright 2008, 2009, 2010 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +#ifndef LSST_QSERV_UTIL_XROOTD_H +#define LSST_QSERV_UTIL_XROOTD_H + +// xrootd.h : consolidates xrootd/lower-level helper functions (i.e., +// dealing with xrootd URLs) + +// Third-party headers +#include + +/// &&& file seems unused, delete if possible + +namespace lsst::qserv::util { + +std::string makeUrl(char const* hostport, char const* typeStr, int chunk); +std::string makeUrl(char const* hostport, char const* typeStr, std::string const& s, char mode = 0); +std::string makeUrl(char const* hostport, std::string const& path); + +} // namespace lsst::qserv::util + +#endif // LSST_QSERV_UTIL_XROOTD_H diff --git a/src/wbase/MsgProcessor.h b/src/wbase/MsgProcessor.h deleted file mode 100644 index 8b48de7ec4..0000000000 --- a/src/wbase/MsgProcessor.h +++ /dev/null @@ -1,64 +0,0 @@ - -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2011-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// MsgProcessor.h -#ifndef LSST_QSERV_WBASE_MSG_PROCESSOR_H -#define LSST_QSERV_WBASE_MSG_PROCESSOR_H - -// System headers -#include -#include - -// Third party headers -#include "nlohmann/json.hpp" - -// Forward declarations -namespace lsst::qserv::wbase { -class Task; -struct TaskSelector; -class WorkerCommand; -} // namespace lsst::qserv::wbase - -namespace lsst::qserv::wbase { - -/// MsgProcessor implementations handle incoming Task objects. -struct MsgProcessor { // &&& delete file if possible - virtual ~MsgProcessor() {} - - /// Process a group of query processing tasks. - virtual void processTasks(std::vector> const& tasks) = 0; // &&& delete - - /// Process a managememt command - virtual void processCommand(std::shared_ptr const& command) = 0; // &&& can this be deleted - - /** - * Retreive the status of queries being processed by the worker. - * @param taskSelector Task selection criterias. - * @return a JSON representation of the object's status for the monitoring - */ - virtual nlohmann::json statusToJson(wbase::TaskSelector const& taskSelector) = 0; // &&& can this be deleted -}; - -} // namespace lsst::qserv::wbase - -#endif // LSST_QSERV_WBASE_MSG_PROCESSOR_H diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 7a4ca7233f..03251465ee 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -280,8 +280,11 @@ std::vector Task::createTasksForChunk( std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { QueryId qId = ujData->getQueryId(); UberJobId ujId = ujData->getUberJobId(); + CzarIdType czId = ujData->getCzarId(); - UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + //&&&UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); + UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); string funcN(__func__); funcN += " QID=" + to_string(qId) + " "; @@ -346,19 +349,35 @@ std::vector Task::createTasksForChunk( if (fragSubchunkIds.empty()) { bool const noSubchunks = false; int const subchunkId = -1; + /* &&& auto task = Task::Ptr(new Task( ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, userQueryInfo, templateId, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, - maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); + maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, queryStats, + resultsHttpPort)); + */ + auto task = Task::Ptr(new Task( + ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, templateId, + noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats, resultsHttpPort)); + vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; + auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, + fragmentNumber, templateId, hasSubchunks, subchunkId, + jdQuerySpecDb, scanInfo, scanInteractive, + maxTableSizeMb, fragSubTables, fragSubchunkIds, + sendChannel, queryStats, resultsHttpPort)); + /* &&& auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, userQueryInfo, templateId, hasSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, fragSubTables, - fragSubchunkIds, sendChannel, resultsHttpPort)); + fragSubchunkIds, sendChannel, queryStats, + resultsHttpPort)); + */ vect.push_back(task); } } diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 4c20cebed0..12cd5c5dcb 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -175,7 +175,7 @@ class Task : public util::CommandForThreadPool { mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, std::shared_ptr const& queriesAndChunks); - /// Create Tasks needed to run unit tests. + /// Create Tasks needed to run unit tests. static std::vector createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, std::shared_ptr const& sendChannel, int maxTableSizeMb, diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 3cf22aacad..4d6db29e29 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -113,7 +113,7 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount workerIdStr = _foreman->chunkInventory()->id(); } else { workerIdStr = "dummyWorkerIdStr"; - LOGS(_log, LOG_LVL_INFO, funcN << " _foreman was null, which should only happen in unit tests"); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " _foreman was null, which should only happen in unit tests"); } json request = {{"version", http::MetaModule::version}, @@ -333,4 +333,106 @@ UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { return newPtr; } +void UberJobData::cancelAllTasks() { + LOGS(_log, LOG_LVL_INFO, cName(__func__)); + if (_cancelled.exchange(true) == false) { + lock_guard lg(_ujTasksMtx); + for (auto const& task : _ujTasks) { + task->cancel(); + } + } +} + +string UJTransmitCmd::cName(const char* funcN) const { + stringstream os; + os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " qId=" << _queryId << " ujId=" << _uberJobId; + return os.str(); +} + +void UJTransmitCmd::action(util::CmdData* data) { + _attemptCount++; + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&& start attempt=" << _attemptCount); + auto ujPtr = _ujData.lock(); + if (ujPtr == nullptr || ujPtr->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " UberJob was cancelled " << _attemptCount); + } + http::Client client(_method, _url, _requestStr, _headers); + bool transmitSuccess = false; + try { + json const response = client.readAsJson(); + if (0 != response.at("success").get()) { + transmitSuccess = true; + _selfPtr.reset(); // clear so this can be deleted. + } else { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " Transmit success == 0"); + // There's no point in re-sending as the czar got the message and didn't like + // it. + // &&& maybe add this czId+ujId to a list of failed uberjobs that can be put + // &&& status return??? Probably overkill. + _selfPtr.reset(); // clear so this can be deleted. + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&& start d except"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " " + _requestContext + " failed, ex: " + ex.what()); + } + + if (!transmitSuccess) { + auto sPtr = _selfPtr; + if (_foreman != nullptr && sPtr != nullptr) { + // Do not reset _selfPtr as re-queuing may be needed several times. + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no response for transmit, putting on failed transmit queue."); + auto wCzInfo = _foreman->getWCzarInfoMap()->getWCzarInfo(_czarId); + // This will check if the czar is believed to be alive and try the queue the query to be tried again + // at a lower priority. It it thinks the czar is dead, it will throw it away. + // TODO:UJ &&& I have my doubts about this as a reconnected czar may go down in flames + // &&& as it is hit with thousands of these. + // &&& Alternate plan, set a flag in the status message response (WorkerQueryStatusData) + // &&& indicates some messages failed. When the czar sees the flag, it'll request a + // &&& message from the worker that contains all of the failed transmit data and handle + // &&& that. All of these failed transmits should fit in a single message. + if (wCzInfo->checkAlive(CLOCK::now())) { + auto wPool = _foreman->getWPool(); + if (wPool != nullptr) { + Ptr replacement = duplicate(); + _selfPtr.reset(); + if (replacement != nullptr) { + wPool->queCmd(replacement, 2); + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " replacement was null"); + } + } else{ + // No thread pool, should only be possible in unit tests. + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no wPool"); + _selfPtr.reset(); + return; + } + } + } else { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _selfPtr was null, assuming job killed."); + _selfPtr.reset(); // In case _foreman is null. + } + } + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&& start end"); +} + +void UJTransmitCmd::kill() { + string const funcN("UJTransmitCmd::kill"); + LOGS(_log, LOG_LVL_WARN, funcN); + auto sPtr = _selfPtr; + _selfPtr.reset(); + if (sPtr == nullptr) { return; } + // &&& TODO:UJ Is there anything that should be done here??? +} + +UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { + auto ujD = _ujData.lock(); + if (ujD == nullptr) { + return nullptr; + } + Ptr newPtr = create(_foreman, ujD, _method, _headers, _url, _requestContext, _requestStr); + newPtr->_attemptCount = _attemptCount; + return newPtr; + +} + } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index f5626acd67..db3d084e1a 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -38,6 +38,7 @@ #include "qmeta/types.h" #include "util/QdispPool.h" #include "wbase/SendChannel.h" +#include "util/InstanceCount.h" namespace lsst::qserv { @@ -126,6 +127,7 @@ class UberJobData : public std::enable_shared_from_this { /// Add the tasks defined in the UberJob to this UberJobData object. void addTasks(std::vector> const& tasks) { + std::lock_guard tLg(_ujTasksMtx); _ujTasks.insert(_ujTasks.end(), tasks.begin(), tasks.end()); } @@ -139,6 +141,11 @@ class UberJobData : public std::enable_shared_from_this { std::string getIdStr() const { return _idStr; } std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } + bool getCancelled() const { return _cancelled; } + + /// &&& doc + void cancelAllTasks(); + private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, int czarPort, uint64_t queryId, int rowLimit, uint64_t maxTableSizeBytes, @@ -154,6 +161,10 @@ class UberJobData : public std::enable_shared_from_this { std::string const& url_, std::string const& requestContext_, std::string const& requestStr_); + /// &&& doc + void _queueUJResponse(http::Method method_, std::vector const& headers_, std::string const& url_, std::string const& requestContext_, std::string const& requestStr_); + + UberJobId const _uberJobId; std::string const _czarName; qmeta::CzarId const _czarId; diff --git a/src/wcontrol/Foreman.cc b/src/wcontrol/Foreman.cc index f990667359..75f557e7f6 100644 --- a/src/wcontrol/Foreman.cc +++ b/src/wcontrol/Foreman.cc @@ -177,6 +177,8 @@ Foreman::~Foreman() { _httpServer->stop(); } +//&&& wpublish::QueryStatistics::Ptr Foreman::addQueryId(QueryId qId) { return _queries->addQueryId(qId); } + void Foreman::processTasks(vector const& tasks) { std::vector cmds; _queries->addTasks(tasks, cmds); diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 7447cff8bc..184b6c6f87 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -187,7 +187,6 @@ struct Fixture { {"tblScanRating", mInfo.scanRating}}; chunkScanTables.push_back(move(cst)); - auto& jsFragments = jsJobMsg["queryFragments"]; /* &&& if (chunkQuerySpec.nextFragment.get()) { @@ -211,8 +210,8 @@ struct Fixture { for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); } - _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, - chunkQuerySpec.queries); + _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, + chunkQuerySpec.subChunkIds, chunkQuerySpec.queries); } */ nlohmann::json jsFrag = {{"resultTable", mInfo.resultName}, @@ -282,22 +281,20 @@ BOOST_AUTO_TEST_CASE(Simple) { MsgInfo mInfo; auto msgJson = newTaskJson(mInfo); shared_ptr sendC(SendChannel::newNopChannel()); - auto sc = FileChannelShared::create(sendC, mInfo.czarId); + auto sChannel = FileChannelShared::create(sendC, mInfo.czarId); FakeBackend::Ptr backend = make_shared(); shared_ptr crm = ChunkResourceMgr::newMgr(backend); SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); auto const queries = queriesAndChunks(); auto ujData = lsst::qserv::wbase::UberJobData::create(mInfo.uberJobId, mInfo.czarName, mInfo.czarId, - mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); lsst::qserv::proto::ScanInfo scanInfo; scanInfo.scanRating = mInfo.scanRating; scanInfo.infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - vector taskVect = Task::createTasksForChunk( - ujData, *msgJson, sc, scanInfo, - mInfo.scanInteractive, mInfo.maxTableSize, - crm, - newMySqlConfig(), sqlConnMgr, - queries); + vector taskVect = + Task::createTasksForChunk(ujData, *msgJson, sChannel, scanInfo, mInfo.scanInteractive, + mInfo.maxTableSize, crm, newMySqlConfig(), sqlConnMgr, queries); Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); @@ -340,20 +337,17 @@ BOOST_AUTO_TEST_CASE(Output) { SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); auto const queries = queriesAndChunks(); auto ujData = lsst::qserv::wbase::UberJobData::create(mInfo.uberJobId, mInfo.czarName, mInfo.czarId, - mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); lsst::qserv::proto::ScanInfo scanInfo; scanInfo.scanRating = mInfo.scanRating; scanInfo.infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - vector taskVect = Task::createTasksForChunk( - ujData, *msgJson, sc, scanInfo, - mInfo.scanInteractive, mInfo.maxTableSize, - crm, - newMySqlConfig(), sqlConnMgr, - queries); + vector taskVect = + Task::createTasksForChunk(ujData, *msgJson, sc, scanInfo, mInfo.scanInteractive, + mInfo.maxTableSize, crm, newMySqlConfig(), sqlConnMgr, queries); Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); - } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wpublish/QueriesAndChunks.h b/src/wpublish/QueriesAndChunks.h index a3d89e1e64..757267fbc7 100644 --- a/src/wpublish/QueriesAndChunks.h +++ b/src/wpublish/QueriesAndChunks.h @@ -202,6 +202,13 @@ class QueriesAndChunks { /// @see _addQueryId QueryStatistics::Ptr addQueryId(QueryId qId, CzarIdType czarId); + /// Return the statistics for a user query, creating if needed. + /// Since it is possible to get messages out of order, there + /// are several case where something like a cancellation + /// message arrives before any tasks have been created. + /// @see getStats() + QueryStatistics::Ptr addQueryId(QueryId qId, CzarIdType czarId); + void addTask(wbase::Task::Ptr const& task); void addTasks(std::vector const& tasks, std::vector& cmds); void queuedTask(wbase::Task::Ptr const& task); diff --git a/src/wpublish/QueryStatistics.cc b/src/wpublish/QueryStatistics.cc index 2ca96d7f37..bc06eea581 100644 --- a/src/wpublish/QueryStatistics.cc +++ b/src/wpublish/QueryStatistics.cc @@ -189,6 +189,13 @@ QueryStatistics::SchedTasksInfoMap QueryStatistics::getSchedulerTasksInfoMap() { return _taskSchedInfoMap; } +/* &&& +void QueryStatistics::touch(TIMEPOINT const now) { + lock_guard lock(_qStatsMtx); + _touched = now; +} +*/ + void QueryStatistics::addTask(TIMEPOINT const now) { lock_guard lock(_qStatsMtx); _touched = now; diff --git a/src/wpublish/QueryStatistics.h b/src/wpublish/QueryStatistics.h index 5fd24ff003..4a366ad5a6 100644 --- a/src/wpublish/QueryStatistics.h +++ b/src/wpublish/QueryStatistics.h @@ -41,6 +41,7 @@ #include "global/intTypes.h" #include "wbase/Task.h" #include "wsched/SchedulerBase.h" +#include "util/InstanceCount.h" //&&& namespace lsst::qserv::wbase { class UserQueryInfo; @@ -95,6 +96,7 @@ class QueryStatistics { void addTaskTransmit(double timeSeconds, int64_t bytesTransmitted, int64_t rowsTransmitted, double bufferFillSecs); + //&&&void touch(TIMEPOINT const now); void addTask(TIMEPOINT const now); void addTaskRunning(TIMEPOINT const now); bool addTaskCompleted(TIMEPOINT const now, double const taskDuration); diff --git a/src/xrdreq/QueryManagementAction.h b/src/xrdreq/QueryManagementAction.h deleted file mode 100644 index ec5ff91588..0000000000 --- a/src/xrdreq/QueryManagementAction.h +++ /dev/null @@ -1,97 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H -#define LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H - -// System headers -#include -#include -#include -#include -#include - -// Qserv headers -#include "global/intTypes.h" -#include "proto/worker.pb.h" - -namespace lsst::qserv::xrdreq { - -/** - * Class QueryManagementAction is an interface for managing query completion/cancellation - * at all Qserv workers that are connected as "publishers" to the XROOTD redirector. - */ -// &&& need to get the same functionality using json messages, and not in xrdreq. -class QueryManagementAction : public std::enable_shared_from_this { -public: - /// The reponse type represents errors reported by the workers, where worker - /// names are the keys. And the values are the error messages. Empty strings - /// indicate the succesful completion of the requests. - using Response = std::map; - - /// The callback function type to be used for notifications on the operation completion. - using CallbackType = std::function; - - /** - * The front-end method for initiating the operation at all workers. - * - * @note The only way to track the completion of the requests sent via - * this interface is by providing the callback function. The request delivery - * is not guaranteeded in case if the XROOTD/SSI network will be clogged by - * the heavy traffic. It's safe to call the same operation many times if needed. - * - * @param xrootdFrontendUrl A location of the XROOTD redirector. - * @param op An operation be initiated at the workers. - * @param onFinish The optional callback to be fired upon the completion of - * the requested operation. - * - * @throws std::runtime_error For failures encountered when connecting to - * the manager or initiating the requesed operation. - */ - static void notifyAllWorkers(std::string const& xrootdFrontendUrl, proto::QueryManagement::Operation op, - uint32_t czarId, QueryId queryId, CallbackType onFinish = nullptr); - - QueryManagementAction(QueryManagementAction const&) = delete; - QueryManagementAction& operator=(QueryManagementAction const&) = delete; - virtual ~QueryManagementAction(); - -private: - QueryManagementAction(); - - /** - * The actual implementation of the request processor. - * @see QueryManagementAction::notifyAllWorkers() - */ - void _notifyAllWorkers(std::string const& xrootdFrontendUrl, proto::QueryManagement::Operation op, - uint32_t czarId, QueryId queryId, CallbackType onFinish); - - /// The collection of worker responses. - Response _response; - - /// The counter will get incremented as worker responses will be received. - /// User-provided callback function (if any) will be called when all requests - /// will finish (succeed or fail). - std::atomic _numWorkerRequestsFinished{0}; -}; - -} // namespace lsst::qserv::xrdreq - -#endif // LSST_QSERV_XRDREQ_QUERY_MANAGEMENT_ACTION_H diff --git a/src/xrdsvc/ChannelStream.h b/src/xrdsvc/ChannelStream.h deleted file mode 100644 index 61c8777e7e..0000000000 --- a/src/xrdsvc/ChannelStream.h +++ /dev/null @@ -1,75 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDSVC_CHANNELSTREAM_H -#define LSST_QSERV_XRDSVC_CHANNELSTREAM_H - -// System headers -#include -#include -#include -#include - -// qserv headers -#include "xrdsvc/StreamBuffer.h" - -// Third-party headers -#include "XrdSsi/XrdSsiErrInfo.hh" // required by XrdSsiStream -#include "XrdSsi/XrdSsiStream.hh" - -namespace lsst::qserv::xrdsvc { - -/// ChannelStream is an implementation of an XrdSsiStream that accepts -/// SendChannel streamed data. -class ChannelStream : public XrdSsiStream { // &&& delete -public: - ChannelStream(); - virtual ~ChannelStream(); - - /// Push in a data packet - void append(StreamBuffer::Ptr const &StreamBuffer, bool last); - - /// Empty _msgs, calling StreamBuffer::Recycle() where needed. - void clearMsgs(); - - /// Pull out a data packet as a Buffer object (called by XrdSsi code) - Buffer *GetBuff(XrdSsiErrInfo &eInfo, int &dlen, bool &last) override; - - bool closed() const { return _closed; } - - uint64_t getSeq() const { return _seq; } - -private: - bool _closed; ///< Closed to new append() calls? - // Can keep a deque of (buf, bufsize) to reduce copying, if needed. - std::deque _msgs; ///< Message queue - std::mutex _mutex; ///< _msgs protection - std::condition_variable _hasDataCondition; ///< _msgs condition - uint64_t const _seq; ///< Unique identifier for this instance. - static std::atomic _sequenceSource; ///< Source of unique identifiers. - std::atomic _appendCount{0}; ///< number of appends - std::atomic _getBufCount{0}; ///< number of buffers -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDSVC_CHANNELSTREAM_H diff --git a/src/xrdsvc/HttpSvc.cc b/src/xrdsvc/HttpSvc.cc index 365f4c0a8b..9694f0533e 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/xrdsvc/HttpSvc.cc @@ -150,7 +150,7 @@ uint16_t HttpSvc::start() { _httpServerPtr->addHandlers( {{"POST", "/querystatus", [self](shared_ptr const& req, shared_ptr const& resp) { - HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "QUERYSTATUS", + HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "/querystatus", http::AuthType::REQUIRED); }}}); _httpServerPtr->start(); diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index b9446ec62d..fcce3157cb 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -36,6 +36,7 @@ #include "http/MetaModule.h" #include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" +#include "http/WorkerQueryStatusData.h" #include "mysql/MySqlUtils.h" #include "protojson/UberJobMsg.h" #include "protojson/WorkerQueryStatusData.h" @@ -248,6 +249,12 @@ json HttpWorkerCzarModule::_handleQueryStatus(std::string const& func) { wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czId); wCzarInfo->czarMsgReceived(CLOCK::now()); + auto const czInfo = wqsData->getCzInfo(); + CzarIdType czId = czInfo->czId; + wcontrol::WCzarInfoMap::Ptr wCzarMap = foreman()->getWCzarInfoMap(); + wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czId); + wCzarInfo->czarMsgReceived(CLOCK::now()); + // For all queryId and czarId items, if the item can't be found, it is simply ignored. Anything that // is missed will eventually be picked up by other mechanisms, such as results being rejected // by the czar. This almost never happen, but the system should respond gracefully. diff --git a/src/xrdsvc/SsiRequest.cc b/src/xrdsvc/SsiRequest.cc deleted file mode 100644 index 1b4ca9aeb5..0000000000 --- a/src/xrdsvc/SsiRequest.cc +++ /dev/null @@ -1,348 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include -#include -#include -#include -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiRequest.hh" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "global/intTypes.h" -#include "global/LogContext.h" -#include "global/ResourceUnit.h" -#include "proto/FrameBuffer.h" -#include "proto/worker.pb.h" -#include "util/InstanceCount.h" -#include "util/HoldTrack.h" -#include "util/Timer.h" -#include "wbase/FileChannelShared.h" -#include "wbase/TaskState.h" -#include "wbase/Task.h" -#include "wconfig/WorkerConfig.h" -#include "wcontrol/Foreman.h" -#include "wcontrol/ResourceMonitor.h" -#include "wpublish/ChunkInventory.h" -#include "xrdsvc/ChannelStream.h" - -namespace proto = lsst::qserv::proto; -namespace wbase = lsst::qserv::wbase; - -namespace { - -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiRequest"); - -} // namespace - -namespace lsst::qserv::xrdsvc { - -SsiRequest::Ptr SsiRequest::newSsiRequest(std::string const& rname, - std::shared_ptr const& foreman) { - auto req = SsiRequest::Ptr(new SsiRequest(rname, foreman)); - req->_selfKeepAlive = req; - return req; -} - -SsiRequest::SsiRequest(std::string const& rname, std::shared_ptr const& foreman) - : _validator(foreman->chunkInventory()->newValidator()), _foreman(foreman), _resourceName(rname) {} - -SsiRequest::~SsiRequest() { - LOGS(_log, LOG_LVL_DEBUG, "~SsiRequest()"); - UnBindRequest(); -} - -void SsiRequest::reportError(std::string const& errStr) { - LOGS(_log, LOG_LVL_WARN, errStr); - replyError(errStr, EINVAL); - ReleaseRequestBuffer(); -} - -uint64_t countLimiter = 0; // LockupDB - -// Step 4 -/// Called by XrdSsi to actually process a request. -void SsiRequest::execute(XrdSsiRequest& req) { - util::Timer t; - LOGS(_log, LOG_LVL_DEBUG, "Execute request, resource=" << _resourceName); - - // We bind this object to the request now. This allows us to respond at any - // time (much simpler). Though the manual forgot to say that all pending - // events will be reflected on a different thread the moment we bind the - // request; the fact allows us to use a mutex to serialize the order of - // initialization and possible early cancellation. We protect this code - // with a mutex gaurd which will be released upon exit. - // - std::lock_guard lock(_finMutex); - BindRequest(req); - - ResourceUnit ru(_resourceName); - - // Make sure the requested resource belongs to this worker - if (!(*_validator)(ru)) { - reportError("WARNING: request to the unowned resource detected:" + _resourceName); - return; - } - - auto const sendChannel = std::make_shared(shared_from_this()); - - // Process the request - switch (ru.unitType()) { - case ResourceUnit::DBCHUNK: { // &&& delete - // Increment the counter of the database/chunk resources in use - _foreman->resourceMonitor()->increment(_resourceName); // &&& TODO:UJ make sure this is implemented elsewhere. - - reportError("&&& DBCHUNK requests are no longer available resource db=" + ru.db() + - " chunkId=" + std::to_string(ru.chunk())); - break; - } - case ResourceUnit::QUERY: { // &&& delete - LOGS(_log, LOG_LVL_DEBUG, "Parsing request details for resource=" << _resourceName); - - reportError("&&& QUERY requests are no longer available"); - - /* &&& - proto::QueryManagement request; - try { - // reqData has the entire request, so we can unpack it without waiting for - // more data. - proto::FrameBufferView view(reqData, reqSize); - view.parse(request); - ReleaseRequestBuffer(); - } catch (proto::FrameBufferError const& ex) { - reportError("Failed to decode a query completion/cancellation command, error: " + - std::string(ex.what())); - break; - } - LOGS(_log, LOG_LVL_DEBUG, - "QueryManagement: op=" << proto::QueryManagement_Operation_Name(request.op()) - << " query_id=" << request.query_id()); - - switch (wconfig::WorkerConfig::instance()->resultDeliveryProtocol()) { - case wconfig::ConfigValResultDeliveryProtocol::XROOT: - case wconfig::ConfigValResultDeliveryProtocol::HTTP: - switch (request.op()) { - case proto::QueryManagement::CANCEL_AFTER_RESTART: - // TODO: locate and cancel the coresponding tasks, remove the tasks - // from the scheduler queues. - wbase::FileChannelShared::cleanUpResultsOnCzarRestart(request.czar_id(), - request.query_id()); - break; - case proto::QueryManagement::CANCEL: - // TODO: locate and cancel the coresponding tasks, remove the tasks - // from the scheduler queues. - wbase::FileChannelShared::cleanUpResults(request.czar_id(), request.query_id()); - break; - case proto::QueryManagement::COMPLETE: - wbase::FileChannelShared::cleanUpResults(request.czar_id(), request.query_id()); - break; - default: - reportError("QueryManagement: op=" + - proto::QueryManagement_Operation_Name(request.op()) + - " is not supported by the current implementation."); - return; - } - break; - default: - throw std::runtime_error("SsiRequest::" + std::string(__func__) + - " unsupported result delivery protocol"); - } - - // Send back the empty response since no info is expected by a caller - // for this type of requests beyond the usual error notifications (if any). - this->reply((char const*)0, 0); - */ - break; - } - default: - reportError("Unexpected unit type '" + std::to_string(ru.unitType()) + - "', resource name: " + _resourceName); - } - // Note that upon exit the _finMutex will be unlocked allowing Finished() - // to actually do something once everything is actually setup. - -} - -/// Called by SSI to free resources. -void SsiRequest::Finished(XrdSsiRequest& req, XrdSsiRespInfo const& rinfo, bool cancel) { // Step 8 - util::HoldTrack::Mark markA(ERR_LOC, "SsiRequest::Finished start"); - if (cancel) { - // Either the czar of xrootd has decided to cancel the Job. - // Try to cancel all of the tasks, if there are any. - for (auto&& wTask : _tasks) { - auto task = wTask.lock(); - if (task != nullptr) { - task->cancel(); - } - } - } - - // This call is sync (blocking). - // client finished retrieving response, or cancelled. - // release response resources (e.g. buf) - // But first we must make sure that request setup completed (i.e execute()) by - // locking _finMutex. - { - std::lock_guard finLock(_finMutex); - // Clean up _stream if it exists and don't add anything new to it either. - _reqFinished = true; - if (_stream != nullptr) { - _stream->clearMsgs(); - } - } - - // This will clear the cyclic dependency: - // FileChannelShared -> ChannelStream -> SsiRequest -> FileChannelShared - // - // TODO: Eliminate xrdsvc::ChannelStream sinve this class seems to be useless - // in the file-based result delivery protocol. - _channelShared.reset(); - - auto keepAlive = freeSelfKeepAlive(); - - // No buffers allocated, so don't need to free. - // We can release/unlink the file now - const char* type = ""; - switch (rinfo.rType) { - case XrdSsiRespInfo::isNone: - type = "type=isNone"; - break; - case XrdSsiRespInfo::isData: - type = "type=isData"; - break; - case XrdSsiRespInfo::isError: - type = "type=isError"; - break; - case XrdSsiRespInfo::isFile: - type = "type=isFile"; - break; - case XrdSsiRespInfo::isStream: - type = "type=isStream"; - break; - case XrdSsiRespInfo::isHandle: - type = "type=isHandle"; - break; - } - - // Decrement the counter of the database/chunk resources in use - ResourceUnit ru(_resourceName); - if (ru.unitType() == ResourceUnit::DBCHUNK) { - _foreman->resourceMonitor()->decrement(_resourceName); - } - - // We can't do much other than close the file. - // It should work (on linux) to unlink the file after we open it, though. - // With the optimizer on '-Og', there was a double free for a SsiRequest. - // The likely cause could be keepAlive being optimized out for being unused. - // The problem has not reoccurred since adding keepAlive to the following - // comment, but having code depend on a comment line is ugly in its own way. - LOGS(_log, LOG_LVL_DEBUG, "RequestFinished " << type << " " << keepAlive.use_count()); -} - -bool SsiRequest::reply(char const* buf, int bufLen) { - Status s = SetResponse(buf, bufLen); - if (s != XrdSsiResponder::wasPosted) { - LOGS(_log, LOG_LVL_ERROR, "DANGER: Couldn't post response of length=" << bufLen); - return false; - } - return true; -} - -bool SsiRequest::replyError(std::string const& msg, int code) { - Status s = SetErrResponse(msg.c_str(), code); - if (s != XrdSsiResponder::wasPosted) { - LOGS(_log, LOG_LVL_ERROR, "DANGER: Couldn't post error response " << msg); - return false; - } - return true; -} - -bool SsiRequest::replyStream(StreamBuffer::Ptr const& sBuf, bool last) { - LOGS(_log, LOG_LVL_DEBUG, "replyStream, checking stream size=" << sBuf->getSize() << " last=" << last); - - // Normally, XrdSsi would call Recycle() when it is done with sBuf, but if this function - // returns false, then it must call Recycle(). Otherwise, the scheduler will likely - // wedge waiting for the buffer to be released. - std::lock_guard finLock(_finMutex); - if (_reqFinished) { - // Finished() was called, give up. - LOGS(_log, LOG_LVL_ERROR, "replyStream called after reqFinished."); - sBuf->Recycle(); - return false; - } - // Create a stream if needed. - if (!_stream) { - _stream = std::make_shared(); - if (SetResponse(_stream.get()) != XrdSsiResponder::Status::wasPosted) { - LOGS(_log, LOG_LVL_WARN, "SetResponse stream failed, calling Recycle for sBuf"); - // SetResponse return value indicates XrdSsi wont call Recycle(). - sBuf->Recycle(); - return false; - } - } else if (_stream->closed()) { - // XrdSsi isn't going to call Recycle if we wind up here. - LOGS(_log, LOG_LVL_ERROR, "Logic error SsiRequest::replyStream called with stream closed."); - sBuf->Recycle(); - return false; - } - // XrdSsi or Finished() will call Recycle(). - LOGS(_log, LOG_LVL_INFO, "SsiRequest::replyStream seq=" << getSeq()); - _stream->append(sBuf, last); - return true; -} - -bool SsiRequest::sendMetadata(const char* buf, int blen) { - Status stat = SetMetadata(buf, blen); - switch (stat) { - case XrdSsiResponder::wasPosted: - return true; - case XrdSsiResponder::notActive: - LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " notActive"); - break; - case XrdSsiResponder::notPosted: - LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " notPosted blen=" << blen); - break; - default: - LOGS(_log, LOG_LVL_ERROR, "failed to " << __func__ << " unkown state blen=" << blen); - } - return false; -} - -SsiRequest::Ptr SsiRequest::freeSelfKeepAlive() { - Ptr keepAlive = std::move(_selfKeepAlive); - return keepAlive; -} - -uint64_t SsiRequest::getSeq() const { - if (_stream == nullptr) return 0; - return _stream->getSeq(); -} - -} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/SsiRequest.h b/src/xrdsvc/SsiRequest.h deleted file mode 100644 index 5850d18bf8..0000000000 --- a/src/xrdsvc/SsiRequest.h +++ /dev/null @@ -1,128 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDSVC_SSIREQUEST_H -#define LSST_QSERV_XRDSVC_SSIREQUEST_H - -// System headers -#include -#include -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiResponder.hh" - -// Qserv headers -#include "global/ResourceUnit.h" -#include "mysql/MySqlConfig.h" -#include "xrdsvc/StreamBuffer.h" - -// Forward declarations -class XrdSsiService; - -namespace lsst::qserv { -namespace wbase { -class FileChannelShared; -class SendChannel; -class Task; -} // namespace wbase -namespace wcontrol { -class Foreman; -} -} // namespace lsst::qserv - -namespace lsst::qserv::xrdsvc { - -class ChannelStream; -class StreamBuffer; - -/// An implementation of XrdSsiResponder that is used by SsiService to provide -/// qserv worker services. The SSI interface encourages such an approach, and -/// object lifetimes are explicitly stated in the documentation which we -/// adhere to using BindRequest() and UnBindRequest() responder methods. -class SsiRequest : public XrdSsiResponder, public std::enable_shared_from_this { // &&& delete if possible -public: - // Smart pointer definitions - - typedef std::shared_ptr ValidatorPtr; - typedef std::shared_ptr Ptr; - - /// Use factory to ensure proper construction for enable_shared_from_this. - static SsiRequest::Ptr newSsiRequest(std::string const& rname, - std::shared_ptr const& processor); - - virtual ~SsiRequest(); - - void execute(XrdSsiRequest& req); - - /** - * Implements the virtual method defined in the base class - * @see XrdSsiResponder::Finished - */ - void Finished(XrdSsiRequest& req, XrdSsiRespInfo const& rinfo, bool cancel = false) override; - - bool isFinished() { return _reqFinished; } - - bool reply(char const* buf, int bufLen); - bool replyError(std::string const& msg, int code); - bool replyStream(StreamBuffer::Ptr const& sbuf, bool last); - - bool sendMetadata(const char* buf, int blen); - - /// Call this to allow object to die after it truly is no longer needed. - /// i.e. It is know Finish() will not be called. - /// NOTE: It is important that any non-static SsiRequest member - /// function make a local copy of the returned pointer so that - /// SsiRequest is guaranteed to live to the end of - /// the function call. - Ptr freeSelfKeepAlive(); - - uint64_t getSeq() const; - -private: - /// Constructor (called by the static factory method newSsiRequest) - SsiRequest(std::string const& rname, std::shared_ptr const& processor); - - /// For internal error reporting - void reportError(std::string const& errStr); - -private: - ValidatorPtr _validator; ///< validates request against what's available - std::shared_ptr const _foreman; ///< actual msg processor - - std::mutex _finMutex; ///< Protects execute() from Finish(), _finished, and _stream - std::atomic _reqFinished{false}; ///< set to true when Finished called - std::string _resourceName; ///< chunk identifier - - std::shared_ptr _stream; - std::shared_ptr _channelShared; ///< Must live before Finished() gets called. - std::vector> _tasks; ///< List of tasks for use in cancellation. - - /// Make sure this object exists until Finish() is called. - /// Make a local copy before calling reset() within and non-static member function. - Ptr _selfKeepAlive; -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDSVC_SSIREQUEST_H From 1e8156edd8d27b584fb5f0743ed12ca03d545c72 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 1 Oct 2024 16:26:34 -0700 Subject: [PATCH 07/15] Added worker believed czar was dead handling. --- src/CMakeLists.txt | 1 + src/ccontrol/UserQuerySelect.cc | 142 ++++++---- src/czar/ActiveWorker.cc | 74 +++-- src/czar/Czar.cc | 1 + src/czar/CzarChunkMap.cc | 15 +- src/czar/CzarRegistry.cc | 9 +- src/czar/CzarRegistry.h | 8 +- src/czar/HttpCzarWorkerModule.cc | 47 +++- src/http/CMakeLists.txt | 2 - src/http/Module.h | 2 - src/http/WorkerQueryStatusData.cc | 437 ----------------------------- src/http/WorkerQueryStatusData.h | 371 ------------------------ src/http/testStatusData.cc | 169 ----------- src/protojson/ScanTableInfo.h | 2 +- src/protojson/UberJobMsg.cc | 18 ++ src/protojson/UberJobMsg.h | 1 - src/protojson/testUberJobMsg.cc | 43 +++ src/qana/QueryMapping.h | 2 + src/qdisp/CMakeLists.txt | 1 - src/qdisp/Executive.cc | 75 +++-- src/qdisp/Executive.h | 6 +- src/qdisp/JobBase.cc | 54 ---- src/qdisp/JobBase.h | 79 ------ src/qdisp/JobDescription.h | 2 +- src/qdisp/JobQuery.cc | 10 +- src/qdisp/JobQuery.h | 19 +- src/qdisp/SharedResources.h | 64 ----- src/qdisp/UberJob.cc | 64 ++--- src/qdisp/testQDisp.cc | 119 ++++++++ src/qproc/ChunkQuerySpec.h | 3 +- src/qproc/QuerySession.cc | 1 + src/qproc/TaskMsgFactory.h | 75 ----- src/rproc/InfileMerger.cc | 2 +- src/util/xrootd.cc | 91 ------ src/util/xrootd.h | 44 --- src/wbase/FileChannelShared.cc | 10 + src/wbase/Task.cc | 125 +-------- src/wbase/Task.h | 12 + src/wbase/UberJobData.cc | 39 +-- src/wbase/UberJobData.h | 13 +- src/wcontrol/Foreman.cc | 2 - src/wdb/QueryRunner.cc | 1 - src/wdb/testQueryRunner.cc | 76 ++--- src/wdb/testQuerySql.cc | 101 ------- src/wpublish/QueriesAndChunks.cc | 1 + src/wpublish/QueryStatistics.cc | 7 - src/wpublish/QueryStatistics.h | 2 - src/wsched/BlendScheduler.cc | 1 + src/xrdsvc/HttpWorkerCzarModule.cc | 2 +- 49 files changed, 556 insertions(+), 1889 deletions(-) delete mode 100644 src/http/WorkerQueryStatusData.cc delete mode 100644 src/http/WorkerQueryStatusData.h delete mode 100644 src/http/testStatusData.cc delete mode 100644 src/qdisp/JobBase.cc delete mode 100644 src/qdisp/JobBase.h delete mode 100644 src/qdisp/SharedResources.h delete mode 100644 src/qproc/TaskMsgFactory.h delete mode 100644 src/util/xrootd.cc delete mode 100644 src/util/xrootd.h delete mode 100644 src/wdb/testQuerySql.cc diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4b53c9ec46..822a8ae3b5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -89,3 +89,4 @@ add_subdirectory(www) add_subdirectory(xrdlog) add_subdirectory(xrdsvc) + diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 22d23719c1..48ffd39e5b 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -106,6 +106,7 @@ #include "sql/Schema.h" #include "util/Bug.h" #include "util/IterableFormatter.h" +#include "util/Histogram.h" //&&& #include "util/QdispPool.h" #include "util/ThreadPriority.h" #include "qdisp/UberJob.h" @@ -227,7 +228,6 @@ std::string UserQuerySelect::getResultQuery() const { return resultQuery; } -#if 0 // &&& /// Begin running on all chunks added so far. void UserQuerySelect::submit() { auto exec = _executive; @@ -323,7 +323,6 @@ void UserQuerySelect::submit() { _qMetaAddChunks(chunks); } } -#endif //&&& void UserQuerySelect::buildAndSendUberJobs() { // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? @@ -528,10 +527,12 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_DEBUG, funcN << " " << exec->dumpUberJobCounts()); } +util::HistogramRolling histoBuildAndS("&&&uj histoBuildAndS", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + void UserQuerySelect::buildAndSendUberJobs() { - // &&& NEED CODE - this function should check if the worker is DEAD. TODO:UJ + // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); - LOGS(_log, LOG_LVL_DEBUG, funcN << " start"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start " << _uberJobMaxChunks); // Ensure `_monitor()` doesn't do anything until everything is ready. if (!_executive->isReadyToExecute()) { @@ -547,14 +548,16 @@ void UserQuerySelect::buildAndSendUberJobs() { qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); if (unassignedChunksInQuery.empty()) { - LOGS(_log, LOG_LVL_TRACE, funcN << " no unassigned Jobs"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " no unassigned Jobs"); return; } + // Get czar info and the worker contactMap. auto czarPtr = czar::Czar::getCzar(); auto czFamilyMap = czarPtr->getCzarFamilyMap(); auto czChunkMap = czFamilyMap->getChunkMap(_queryDbName); auto czRegistry = czarPtr->getCzarRegistry(); + auto const wContactMap = czRegistry->waitForWorkerContactMap(); if (czChunkMap == nullptr) { LOGS(_log, LOG_LVL_ERROR, funcN << " no map found for queryDbName=" << _queryDbName); @@ -579,72 +582,111 @@ void UserQuerySelect::buildAndSendUberJobs() { // - For failures - If a worker cannot be contacted, that's an uberjob failure. // - uberjob failures (due to communications problems) will result in the uberjob // being broken up into multiple UberJobs going to different workers. - // - The best way to do this is probably to just kill the UberJob and mark all - // Jobs that were in that UberJob as needing re-assignment, and re-running - // the code here. The trick is going to be figuring out which workers are alive. - // Maybe force a fresh lookup from the replicator Registry when an UberJob fails. - map> workerJobMap; + // - If an UberJob fails, the UberJob is killed and all the Jobs it contained + // are flagged as needing re-assignment and this function will be called + // again to put those Jobs in new UberJobs. Correctly re-assigning the + // Jobs requires accurate information from the registry about which workers + // are alive or dead. + struct WInfoAndUJPtr { + using Ptr = shared_ptr; + qdisp::UberJob::Ptr uberJobPtr; + protojson::WorkerContactInfo::Ptr wInf; + }; + map workerJobMap; vector missingChunks; + auto startassign = CLOCK::now(); //&&& // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in - // numerical order. The workers run shared scans in numerical order of chunk id numbers. - // This keeps the number of partially complete UberJobs running on a worker to a minimum, + // numerical order. The workers run shared scans in numerical order of chunkId numbers. + // Numerical order keeps the number of partially complete UberJobs running on a worker to a minimum, // and should minimize the time for the first UberJob on the worker to complete. for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { + + bool const increaseAttemptCount = true; + jqPtr->getDescription()->incrAttemptCount(_executive, increaseAttemptCount); + + // If too many workers are down, there will be a chunk that cannot be found. + // Just continuing should leave jobs `unassigned` with their attempt count + // increased. Either the chunk will be found and jobs assigned, or the jobs' + // attempt count will reach max and the query will be cancelled + auto lambdaMissingChunk = [&](string const& msg) { + missingChunks.push_back(chunkId); + //&&&bool const increaseAttemptCount = true; + //&&&jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); + LOGS(_log, LOG_LVL_ERROR, msg); + }; + auto iter = chunkMapPtr->find(chunkId); if (iter == chunkMapPtr->end()) { - missingChunks.push_back(chunkId); - bool const increaseAttemptCount = true; - jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); - // Assign as many jobs as possible. Any chunks not found will be attempted later. + lambdaMissingChunk(funcN + " No chunkData for=" + to_string(chunkId)); continue; } czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - // TODO:UJ maybe if (targetWorker == nullptr || this worker already tried for this chunk) { - if (targetWorker == nullptr) { - LOGS(_log, LOG_LVL_ERROR, funcN << " No primary scan worker for chunk=" << chunkData->dump()); + // TODO:UJ maybe if (targetWorker == nullptr || ... || this worker already tried for this chunk) { + if (targetWorker == nullptr || targetWorker->isDead()) { + LOGS(_log, LOG_LVL_WARN, + funcN << " No primary scan worker for chunk=" + chunkData->dump() + << ((targetWorker == nullptr) ? " targ was null" : " targ was dead")); // Try to assign a different worker to this job auto workerHasThisChunkMap = chunkData->getWorkerHasThisMapCopy(); bool found = false; for (auto wIter = workerHasThisChunkMap.begin(); wIter != workerHasThisChunkMap.end() && !found; ++wIter) { auto maybeTarg = wIter->second.lock(); - if (maybeTarg != nullptr) { + if (maybeTarg != nullptr && !maybeTarg->isDead()) { targetWorker = maybeTarg; found = true; LOGS(_log, LOG_LVL_WARN, - funcN << " Alternate worker found for chunk=" << chunkData->dump()); + funcN << " Alternate worker=" << targetWorker->getWorkerId() + << " found for chunk=" << chunkData->dump()); } } if (!found) { - // If too many workers are down, there will be a chunk that cannot be found. - // Just continuing should leave jobs `unassigned` with their attempt count - // increased. Either the chunk will be found and jobs assigned, or the jobs' - // attempt count will reach max and the query will be cancelled - // TODO:UJ Needs testing/verification - LOGS(_log, LOG_LVL_ERROR, - funcN << " No primary or alternate worker found for chunk=" << chunkData->dump()); + lambdaMissingChunk(funcN + + " No primary or alternate worker found for chunk=" + chunkData->dump()); continue; } } // Add this job to the appropriate UberJob, making the UberJob if needed. string workerId = targetWorker->getWorkerId(); - auto& ujVect = workerJobMap[workerId]; - if (ujVect.empty() || ujVect.back()->getJobCount() >= _maxChunksPerUberJob) { + WInfoAndUJPtr::Ptr& wInfUJ = workerJobMap[workerId]; + if (wInfUJ == nullptr) { + wInfUJ = make_shared(); + auto iter = wContactMap->find(workerId); + if (iter == wContactMap->end()) { + // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. + // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, + // so maybe return internal error to the user? + throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + workerId); + } + wInfUJ->wInf = iter->second; + } + + if (wInfUJ->uberJobPtr == nullptr) { auto ujId = _uberJobIdSeq++; // keep ujId consistent string uberResultName = _ttn->make(ujId); auto respHandler = make_shared(_infileMerger, uberResultName); auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), ujId, _qMetaCzarId, targetWorker); - ujVect.push_back(uJob); + uJob->setWorkerContactInfo(wInfUJ->wInf); + wInfUJ->uberJobPtr = uJob; + }; + + wInfUJ->uberJobPtr->addJob(jqPtr); + + if (wInfUJ->uberJobPtr->getJobCount() >= _uberJobMaxChunks) { + // Queue the UberJob to be sent to a worker + _executive->addAndQueueUberJob(wInfUJ->uberJobPtr); + + // Clear the pinter so a new UberJob is created later if needed. + wInfUJ->uberJobPtr = nullptr; } - auto& ujVectBack = ujVect.back(); - ujVectBack->addJob(jqPtr); - LOGS(_log, LOG_LVL_DEBUG, - funcN << " ujVectBack{" << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() - << "}"); } + auto endassign = CLOCK::now(); //&&& + std::chrono::duration secsassign = endassign - startassign; // &&& + histoBuildAndS.addEntry(endassign, secsassign.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoBuildAndS.getString("")); if (!missingChunks.empty()) { string errStr = funcN + " a worker could not be found for these chunks "; @@ -655,27 +697,17 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_ERROR, errStr); } - // Add worker contact info to UberJobs. The czar can't do anything without - // the contact map, so it will wait. This should only ever be an issue at startup. - auto const wContactMap = czRegistry->waitForWorkerContactMap(); - LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); - for (auto const& [wIdKey, ujVect] : workerJobMap) { - auto iter = wContactMap->find(wIdKey); - if (iter == wContactMap->end()) { - // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. - // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, - // so maybe return internal error to the user? - throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + wIdKey); - } - auto const& wContactInfo = iter->second; - for (auto const& ujPtr : ujVect) { - ujPtr->setWorkerContactInfo(wContactInfo); - } - _executive->addUberJobs(ujVect); - for (auto const& ujPtr : ujVect) { - _executive->runUberJob(ujPtr); + // Queue unqued UberJobs, these have less than the max number of jobs. + for (auto const& [wIdKey, winfUjPtr] : workerJobMap) { + if (winfUjPtr != nullptr) { + auto& ujPtr = winfUjPtr->uberJobPtr; + if (ujPtr != nullptr) { + _executive->addAndQueueUberJob(ujPtr); + } } } + + LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); } /// Block until a submit()'ed query completes. diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index 2325b014c8..db59c6720f 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -255,7 +255,7 @@ bool ActiveWorker::compareContactInfo(http::WorkerContactInfo const& wcInfo) con return wInfo_->isSameContactInfo(wcInfo); } -void ActiveWorker::setWorkerContactInfo(http::WorkerContactInfo::Ptr const& wcInfo) { +void ActiveWorker::setWorkerContactInfo(protojson::WorkerContactInfo::Ptr const& wcInfo) { LOGS(_log, LOG_LVL_INFO, cName(__func__) << " new info=" << wcInfo->dump()); lock_guard lg(_aMtx); _wqsData->setWInfo(wcInfo); @@ -271,8 +271,9 @@ void ActiveWorker::_changeStateTo(State newState, double secsSinceUpdate, string void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, double maxLifetime) { + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " start"); bool newlyDeadWorker = false; - http::WorkerContactInfo::Ptr wInfo_; + protojson::WorkerContactInfo::Ptr wInfo_; { lock_guard lg(_aMtx); wInfo_ = _wqsData->getWInfo(); @@ -280,14 +281,18 @@ void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double ti LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no WorkerContactInfo"); return; } - double secsSinceUpdate = (wInfo_ == nullptr) ? timeoutDeadSecs : wInfo_->timeSinceRegUpdateSeconds(); + double secsSinceUpdate = wInfo_->timeSinceRegUpdateSeconds(); + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << " wInfo=" << wInfo_->dump() + << " secsSince=" << wInfo_->timeSinceRegUpdateSeconds() + << " secsSinceUpdate=" << secsSinceUpdate); // Update the last time the registry contacted this worker. + // TODO:UJ - This needs to be added to the dashboard. switch (_state) { case ALIVE: { if (secsSinceUpdate >= timeoutAliveSecs) { _changeStateTo(QUESTIONABLE, secsSinceUpdate, cName(__func__)); - // &&& Anything else that should be done here? } break; } @@ -307,7 +312,6 @@ void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double ti _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); } else { // Don't waste time on this worker until the registry has heard from it. - // &&& If it's been a really really long time, maybe delete this entry ??? return; } break; @@ -324,29 +328,13 @@ void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double ti shared_ptr jsWorkerReqPtr; { - lock_guard lg(_aMtx); //&&& needed ??? - lock_guard mapLg(_wqsData->mapMtx); - // Check how many messages are currently being sent to the worker, if at the limit, return - if (_wqsData->qIdDoneKeepFiles.empty() && _wqsData->qIdDoneDeleteFiles.empty() && - _wqsData->qIdDeadUberJobs.empty()) { - return; - } - int tCount = _conThreadCount; - if (tCount > _maxConThreadCount) { - LOGS(_log, LOG_LVL_DEBUG, - cName(__func__) << " not sending message since at max threads " << tCount); - return; - } - // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a // message to send to the worker. jsWorkerReqPtr = _wqsData->serializeJson(maxLifetime); } - // &&& Maybe only send the status message if the lists are not empty ??? - // Start a thread to send the message. (Maybe these should go on the qdisppool? &&&) - // put this in a different function and start the thread.&&&; - //&&& _sendStatusMsg(wInfo_, jsWorkerReqPtr); + // Always send the message as it's a way to inform the worker that this + // czar is functioning and capable of receiving requests. Ptr thisPtr = shared_from_this(); auto sendStatusMsgFunc = [thisPtr, wInfo_, jsWorkerReqPtr](util::CmdData*) { thisPtr->_sendStatusMsg(wInfo_, jsWorkerReqPtr); @@ -354,10 +342,11 @@ void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double ti auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(sendStatusMsgFunc)); auto qdisppool = czar::Czar::getCzar()->getQdispPool(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " queuing message"); qdisppool->queCmd(cmd, 1); } -void ActiveWorker::_sendStatusMsg(http::WorkerContactInfo::Ptr const& wInf, +void ActiveWorker::_sendStatusMsg(protojson::WorkerContactInfo::Ptr const& wInf, std::shared_ptr const& jsWorkerReqPtr) { auto& jsWorkerReq = *jsWorkerReqPtr; auto const method = http::Method::POST; @@ -378,25 +367,30 @@ void ActiveWorker::_sendStatusMsg(http::WorkerContactInfo::Ptr const& wInf, http::Client client(method, url, jsWorkerReq.dump(), headers); bool transmitSuccess = false; string exceptionWhat; + json response; try { - json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read start"); + response = client.readAsJson(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read end"); if (0 != response.at("success").get()) { bool startupTimeChanged = false; - tie(transmitSuccess, startupTimeChanged) = _wqsData->handleResponseJson(response); + startupTimeChanged = _wqsData->handleResponseJson(response); + transmitSuccess = true; if (startupTimeChanged) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " worker startupTime changed, likely rebooted."); // kill all incomplete UberJobs on this worker. czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInf->wId); } } else { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " response success=0"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure response success=0 " << response); } } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + LOGS(_log, LOG_LVL_ERROR, requestContext + " transmit failure, ex: " + ex.what()); exceptionWhat = ex.what(); } if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure"); + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " transmit failure " << jsWorkerReq.dump() << " resp=" << response); } } @@ -411,6 +405,17 @@ void ActiveWorker::addDeadUberJob(QueryId qId, UberJobId ujId) { _wqsData->addDeadUberJob(qId, ujId, now); } +protojson::WorkerContactInfo::Ptr ActiveWorker::getWInfo() const { + std::lock_guard lg(_aMtx); + if (_wqsData == nullptr) return nullptr; + return _wqsData->getWInfo(); +} + +ActiveWorker::State ActiveWorker::getState() const { + std::lock_guard lg(_aMtx); + return _state; +} + string ActiveWorker::dump() const { lock_guard lg(_aMtx); return _dump(); @@ -422,8 +427,13 @@ string ActiveWorker::_dump() const { return os.str(); } -void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, - http::CzarContactInfo::Ptr const& czInfo, +ActiveWorkerMap::ActiveWorkerMap(std::shared_ptr const& czarConfig) + : _timeoutAliveSecs(czarConfig->getActiveWorkerTimeoutAliveSecs()), + _timeoutDeadSecs(czarConfig->getActiveWorkerTimeoutDeadSecs()), + _maxLifetime(czarConfig->getActiveWorkerMaxLifetimeSecs()) {} + +void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap, + protojson::CzarContactInfo::Ptr const& czInfo, std::string const& replicationInstanceId, std::string const& replicationAuthKey) { // Go through wcMap, update existing entries in _awMap, create new entries for those that don't exist, @@ -432,6 +442,7 @@ void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, auto iter = _awMap.find(wcKey); if (iter == _awMap.end()) { auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " AciveWorker created for " << wcKey); _awMap[wcKey] = newAW; if (_czarCancelAfterRestart) { newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); @@ -446,6 +457,7 @@ void ActiveWorkerMap::updateMap(http::WorkerContactInfo::WCMap const& wcMap, // If there is existing information, only host and port values will change. aWorker->setWorkerContactInfo(wcVal); } + aWorker->getWInfo()->setRegUpdateTime(wcVal->getRegUpdateTime()); } } } diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index fdc55a1e16..05735f898d 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -147,6 +147,7 @@ void Czar::_monitor() { // a separate message (see WorkerCzarComIssue) saying it killed everything that this // czar gave it. Upon getting this message from a worker, this czar will reassign // everything it had sent to that worker. + // TODO:UJ How long should queryId's remain on this list? } } diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 7c3e74880c..33afa5dd4f 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -36,6 +36,8 @@ #include "czar/CzarRegistry.h" #include "qmeta/Exceptions.h" #include "util/Bug.h" +#include "util/InstanceCount.h" //&&& +#include "util/Histogram.h" //&&& #include "util/TimeUtils.h" using namespace std; @@ -332,7 +334,7 @@ bool CzarFamilyMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarFamilyMap::_read() start"); // If replacing the map, this may take a bit of time, but it's probably // better to wait for new maps if something changed. - std::lock_guard gLock(_familyMapMtx); + std::lock_guard gLock(_familyMapMtx); // &&& check waiting is really needed qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(_lastUpdateTime); if (_lastUpdateTime == qChunkMap.updateTime) { LOGS(_log, LOG_LVL_DEBUG, @@ -342,9 +344,6 @@ bool CzarFamilyMap::_read() { return false; } - // &&& TODO:UJ Before makeNewMaps(), get a list of workers considered to be alive by - // czar::_activeWorkerMap - // Make the new maps. auto czConfig = cconfig::CzarConfig::instance(); bool usingChunkSize = czConfig->getFamilyMapUsingChunkSize(); @@ -360,9 +359,13 @@ bool CzarFamilyMap::_read() { return true; } +util::HistogramRolling histoMakeNewMaps("&&&uj histoMakeNewMaps", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + std::shared_ptr CzarFamilyMap::makeNewMaps( qmeta::QMetaChunkMap const& qChunkMap, bool usingChunkSize) { // Create new maps. + util::InstanceCount ic("CzarFamilyMap::makeNewMaps&&&"); + auto startMakeMaps = CLOCK::now(); //&&& std::shared_ptr newFamilyMap = make_shared(); // Workers -> Databases map @@ -419,6 +422,10 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( } } + auto endMakeMaps = CLOCK::now(); //&&& + std::chrono::duration secsMakeMaps = endMakeMaps - startMakeMaps; // &&& + histoMakeNewMaps.addEntry(endMakeMaps, secsMakeMaps.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoMakeNewMaps.getString("")); return newFamilyMap; } diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index 2f74c797e2..c0cb013715 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -75,7 +75,6 @@ protojson::WorkerContactInfo::WCMapPtr CzarRegistry::getWorkerContactMap() const return _contactMap; } - void CzarRegistry::_registryUpdateLoop() { auto const method = http::Method::POST; string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + @@ -181,7 +180,6 @@ protojson::WorkerContactInfo::WCMapPtr CzarRegistry::_buildMapFromJson(nlohmann: return wMap; } - bool CzarRegistry::_compareMapContactInfo(protojson::WorkerContactInfo::WCMap const& other) const { VMUTEX_HELD(_cmapMtx); if (_contactMap == nullptr) { @@ -238,16 +236,15 @@ void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) } void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) { - lock_guard lck(_mapMtx); // Add query id to the appropriate list. if (deleteWorkerResults) { - _activeWorkerMap.addToDoneDeleteFiles(qId); + _activeWorkerMap->addToDoneDeleteFiles(qId); } else { - _activeWorkerMap.addToDoneKeepFiles(qId); + _activeWorkerMap->addToDoneKeepFiles(qId); } // With lists updated, send out messages. - _activeWorkerMap.sendActiveWorkersMessages(); + _activeWorkerMap->sendActiveWorkersMessages(); } } // namespace lsst::qserv::czar diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index fde3b935c5..57d1312dd1 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -38,7 +38,6 @@ #include "global/clock_defs.h" #include "util/Mutex.h" - namespace lsst::qserv::cconfig { class CzarConfig; } // namespace lsst::qserv::cconfig @@ -99,9 +98,11 @@ class CzarRegistry { /// Return _contactMap, the object that the returned pointer points to is /// constant and no attempts should be made to change it. This /// function will wait forever for a valid contact map to be ready. - http::WorkerContactInfo::WCMapPtr waitForWorkerContactMap() const; + protojson::WorkerContactInfo::WCMapPtr waitForWorkerContactMap() const; - /// &&& doc + /// Send all live workers the `WorkerQueryStatusData` message for + /// that worker. This may result in the worker sending back the + /// `WorkerCzarComIssue` message if there were communication problems. void sendActiveWorkersMessages(); /// Add the query id to the list of queries to end on workers and @@ -114,7 +115,6 @@ class CzarRegistry { CzarRegistry(std::shared_ptr const& czarConfig, std::shared_ptr const& activeWorkerMap); - /// This function will keep periodically updating Czar's info in the Replication System's Registry /// until _loop is set to false. /// Communications problems are logged but ignored. This should probably change. diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index de6fd92e17..9b1f5b783e 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -231,7 +231,7 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { // Parse and verify the json message and then kill the UberJob. json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // See qdisp::UberJob::runUberJob() for json message construction. + // See qdisp::UberJob::runUberJob() for json message construction. &&& string const targetWorkerId = body().required("workerid"); string const czarName = body().required("czar"); qmeta::CzarId const czarId = body().required("czarid"); @@ -261,16 +261,20 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { "HttpCzarWorkerModule::_handleJobError received " << iaEx.what() << " js=" << body().objJson); jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; } + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobError end"); return jsRet; } json HttpCzarWorkerModule::_handleJobReady(string const& func) { + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady start"); // Metadata-only responses for the file-based protocol should not have any data // Parse and verify the json message and then have the uberjob import the file. json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // See qdisp::UberJob::runUberJob() for json message construction. + // &&& TODO:UJ file response - move construction and parsing + // &&& TODO:UJ to a class so it can be added to WorkerCzarComIssue + // See qdisp::UberJob::runUberJob() for json message construction. &&& string const targetWorkerId = body().required("workerid"); string const czarName = body().required("czar"); qmeta::CzarId const czarId = body().required("czarid"); @@ -301,6 +305,45 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { "HttpCzarWorkerModule::_handleJobReady received " << iaEx.what() << " js=" << body().objJson); jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; } + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady end"); + return jsRet; +} + +json HttpCzarWorkerModule::_handleWorkerCzarComIssue(string const& func) { + LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleWorkerCzarComIssue start"); + // Parse and verify the json message and then deal with the problems. + json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + string const replicationInstanceId = cconfig::CzarConfig::instance()->replicationInstanceId(); + string const replicationAuthKey = cconfig::CzarConfig::instance()->replicationAuthKey(); + auto const& jsReq = body().objJson; + auto wccIssue = protojson::WorkerCzarComIssue::createFromJson(jsReq, replicationInstanceId, + replicationAuthKey); + + auto wId = wccIssue->getWorkerInfo()->wId; + if (wccIssue->getThoughtCzarWasDead()) { + LOGS(_log, LOG_LVL_WARN, + "HttpCzarWorkerModule::_handleWorkerCzarComIssue worker=" + << wId << " thought czar was dead and killed related uberjobs."); + + // Find all incomplete UberJobs with this workerId and re-assign them. + // Use a copy to avoid mutex issues. + auto execMap = czar::Czar::getCzar()->getExecMapCopy(); + for (auto const& [exKey, execWeak] : execMap) { + auto execPtr = execWeak.lock(); + if (execPtr == nullptr) continue; + execPtr->killIncompleteUberJobsOnWorker(wId); + } + } + jsRet = wccIssue->serializeResponseJson(); + LOGS(_log, LOG_LVL_TRACE, "HttpCzarWorkerModule::_handleWorkerCzarComIssue jsRet=" << jsRet.dump()); + + } catch (std::invalid_argument const& iaEx) { + LOGS(_log, LOG_LVL_ERROR, + "HttpCzarWorkerModule::_handleWorkerCzarComIssue received " << iaEx.what() + << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; + } return jsRet; } diff --git a/src/http/CMakeLists.txt b/src/http/CMakeLists.txt index 61097f9f22..454d4ab88f 100644 --- a/src/http/CMakeLists.txt +++ b/src/http/CMakeLists.txt @@ -19,7 +19,6 @@ target_sources(http PRIVATE RequestBodyJSON.cc RequestQuery.cc Url.cc - WorkerQueryStatusData.cc ) target_link_libraries(http PUBLIC @@ -52,6 +51,5 @@ http_tests( testAsyncReq testRequestBodyJSON testRequestQuery - testStatusData testUrl ) diff --git a/src/http/Module.h b/src/http/Module.h index 2cc5e42d95..6bd149fb16 100644 --- a/src/http/Module.h +++ b/src/http/Module.h @@ -92,8 +92,6 @@ class Module : public BaseModule { */ virtual void sendResponse(std::string const& content, std::string const& contentType) = 0; - std::string authKey() const { return _authKey; } - private: /** * Pull the raw request body and translate it into a JSON object. diff --git a/src/http/WorkerQueryStatusData.cc b/src/http/WorkerQueryStatusData.cc deleted file mode 100644 index e9524a26e1..0000000000 --- a/src/http/WorkerQueryStatusData.cc +++ /dev/null @@ -1,437 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "http/WorkerQueryStatusData.h" - -// System headers -#include - -// Qserv headers -#include "http/Client.h" -#include "http/MetaModule.h" -#include "http/RequestBodyJSON.h" -#include "util/common.h" - -// LSST headers -#include "lsst/log/Log.h" - -using namespace std; -using namespace nlohmann; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.http.WorkerQueryStatusData"); -} // namespace - -namespace lsst::qserv::http { - -json CzarContactInfo::serializeJson() const { - json jsCzar; - jsCzar["name"] = czName; - jsCzar["id"] = czId; - jsCzar["management-port"] = czPort; - jsCzar["management-host-name"] = czHostName; - jsCzar["czar-startup-time"] = czStartupTime; - return jsCzar; -} - -CzarContactInfo::Ptr CzarContactInfo::createFromJson(nlohmann::json const& czJson) { - try { - auto czName_ = RequestBodyJSON::required(czJson, "name"); - auto czId_ = RequestBodyJSON::required(czJson, "id"); - auto czPort_ = RequestBodyJSON::required(czJson, "management-port"); - auto czHostName_ = RequestBodyJSON::required(czJson, "management-host-name"); - auto czStartupTime_ = RequestBodyJSON::required(czJson, "czar-startup-time"); - return create(czName_, czId_, czPort_, czHostName_, czStartupTime_); - //&&& return create(czName_, czId_, czPort_, czHostName_); - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("CzarContactInfo::createJson invalid ") << exc.what()); - } - return nullptr; -} - -std::string CzarContactInfo::dump() const { - stringstream os; - //&&& os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << - //czHostName; - os << "czName=" << czName << " czId=" << czId << " czPort=" << czPort << " czHostName=" << czHostName - << " czStartupTime=" << czStartupTime; - return os.str(); -} - -json WorkerContactInfo::serializeJson() const { - lock_guard lg(_rMtx); - return _serializeJson(); -} - -json WorkerContactInfo::_serializeJson() const { - json jsWorker; - jsWorker["id"] = wId; - jsWorker["host"] = _wHost; - jsWorker["management-host-name"] = _wManagementHost; - jsWorker["management-port"] = _wPort; - jsWorker["w-startup-time"] = _wStartupTime; - return jsWorker; -} - -WorkerContactInfo::Ptr WorkerContactInfo::createFromJsonRegistry(string const& wId_, - nlohmann::json const& regJson) { - try { - auto wHost_ = RequestBodyJSON::required(regJson, "host-addr"); - auto wManagementHost_ = RequestBodyJSON::required(regJson, "management-host-name"); - auto wPort_ = RequestBodyJSON::required(regJson, "management-port"); - auto updateTimeInt = RequestBodyJSON::required(regJson, "update-time-ms"); - TIMEPOINT updateTime_ = TIMEPOINT(chrono::milliseconds(updateTimeInt)); - - return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); - } - return nullptr; -} - -WorkerContactInfo::Ptr WorkerContactInfo::createFromJsonWorker(nlohmann::json const& wJson, - TIMEPOINT updateTime_) { - try { - auto wId_ = RequestBodyJSON::required(wJson, "id"); - auto wHost_ = RequestBodyJSON::required(wJson, "host"); - auto wManagementHost_ = RequestBodyJSON::required(wJson, "management-host-name"); - auto wPort_ = RequestBodyJSON::required(wJson, "management-port"); - - return create(wId_, wHost_, wManagementHost_, wPort_, updateTime_); - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("CWorkerContactInfo::createJson invalid ") << exc.what()); - } - return nullptr; -} - -string WorkerContactInfo::dump() const { - lock_guard lg(_rMtx); - return _dump(); -} - -string WorkerContactInfo::_dump() const { - stringstream os; - os << "workerContactInfo{" - << "id=" << wId << " host=" << _wHost << " mgHost=" << _wManagementHost << " port=" << _wPort << "}"; - return os.str(); -} - -shared_ptr WorkerQueryStatusData::serializeJson(double maxLifetime) { - // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a - // message to send to the worker. - auto now = CLOCK::now(); - shared_ptr jsWorkerReqPtr = make_shared(); - json& jsWorkerR = *jsWorkerReqPtr; - jsWorkerR["version"] = http::MetaModule::version; - jsWorkerR["instance_id"] = _replicationInstanceId; - jsWorkerR["auth_key"] = _replicationAuthKey; - jsWorkerR["czar"] = _czInfo->serializeJson(); - { - lock_guard lgI(_infoMtx); - if (_wInfo != nullptr) { - jsWorkerR["worker"] = _wInfo->serializeJson(); - } else { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo is null"); - } - } - - // Note, old elements in the maps will be deleted after being added to the message - // to keep the czar from keeping track of these forever. - addListsToJson(jsWorkerR, now, maxLifetime); - if (czarCancelAfterRestart) { - jsWorkerR["czarrestart"] = true; - lock_guard mapLg(mapMtx); - jsWorkerR["czarrestartcancelczid"] = czarCancelAfterRestartCzId; - jsWorkerR["czarrestartcancelqid"] = czarCancelAfterRestartQId; - } else { - jsWorkerR["czarrestart"] = false; - } - - return jsWorkerReqPtr; -} - -void WorkerQueryStatusData::addListsToJson(json& jsWR, TIMEPOINT tmMark, double maxLifetime) { - jsWR["qiddonekeepfiles"] = json::array(); - jsWR["qiddonedeletefiles"] = json::array(); - jsWR["qiddeaduberjobs"] = json::array(); - lock_guard mapLg(mapMtx); - { - auto& jsDoneKeep = jsWR["qiddonekeepfiles"]; - auto iterDoneKeep = qIdDoneKeepFiles.begin(); - while (iterDoneKeep != qIdDoneKeepFiles.end()) { - auto qId = iterDoneKeep->first; - jsDoneKeep.push_back(qId); - auto tmTouched = iterDoneKeep->second; - double ageSecs = std::chrono::duration(tmMark - tmTouched).count(); - if (ageSecs > maxLifetime) { - iterDoneKeep = qIdDoneKeepFiles.erase(iterDoneKeep); - } else { - ++iterDoneKeep; - } - } - } - { - auto& jsDoneDelete = jsWR["qiddonedeletefiles"]; - auto iterDoneDelete = qIdDoneDeleteFiles.begin(); - while (iterDoneDelete != qIdDoneDeleteFiles.end()) { - auto qId = iterDoneDelete->first; - jsDoneDelete.push_back(qId); - auto tmStamp = iterDoneDelete->second; - double ageSecs = std::chrono::duration(tmMark - tmStamp).count(); - if (ageSecs > maxLifetime) { - iterDoneDelete = qIdDoneDeleteFiles.erase(iterDoneDelete); - } else { - ++iterDoneDelete; - } - } - } - { - auto& jsDeadUj = jsWR["qiddeaduberjobs"]; - auto iterDeadUjQid = qIdDeadUberJobs.begin(); - while (iterDeadUjQid != qIdDeadUberJobs.end()) { - TIMEPOINT youngestTm = TIMEPOINT::max(); // need to find the youngest - auto qId = iterDeadUjQid->first; - auto& ujIdMap = iterDeadUjQid->second; - - json jsQidUj = {{"qid", qId}, {"ujids", json::array()}}; - auto& jsUjIds = jsQidUj["ujids"]; - - auto iterUjId = ujIdMap.begin(); - bool addedUjId = false; - - while (iterUjId != ujIdMap.end()) { - UberJobId ujId = iterUjId->first; - auto tmStamp = iterUjId->second; - if (tmStamp < youngestTm) { - youngestTm = tmStamp; - } - - jsUjIds.push_back(ujId); - addedUjId = true; - double ageSecs = std::chrono::duration(tmMark - tmStamp).count(); - if (ageSecs > maxLifetime) { - iterUjId = ujIdMap.erase(iterUjId); - } else { - ++iterUjId; - } - } - - if (addedUjId) { - jsDeadUj.push_back(jsQidUj); - } - - // If the youngest element was too old, delete the map. - if (ujIdMap.empty() || std::chrono::duration(tmMark - youngestTm).count() > maxLifetime) { - iterDeadUjQid = qIdDeadUberJobs.erase(iterDeadUjQid); - } else { - ++iterDeadUjQid; - } - } - } -} - -WorkerQueryStatusData::Ptr WorkerQueryStatusData::createFromJson(nlohmann::json const& jsWorkerReq, - std::string const& replicationInstanceId_, - std::string const& replicationAuthKey_, - TIMEPOINT updateTm) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& a"); - try { - if (jsWorkerReq["version"] != http::MetaModule::version) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson bad version"); - return nullptr; - } - - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& b"); - auto czInfo_ = CzarContactInfo::createFromJson(jsWorkerReq["czar"]); - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& c"); - auto wInfo_ = WorkerContactInfo::createFromJsonWorker(jsWorkerReq["worker"], updateTm); - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& d"); - if (czInfo_ == nullptr || wInfo_ == nullptr) { - LOGS(_log, LOG_LVL_ERROR, - "WorkerQueryStatusData::createJson czar or worker info could not be parsed in " - << jsWorkerReq); - } - auto wqsData = - WorkerQueryStatusData::create(wInfo_, czInfo_, replicationInstanceId_, replicationAuthKey_); - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& e"); - wqsData->parseLists(jsWorkerReq, updateTm); - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::createJson &&& end"); - bool czarRestart = RequestBodyJSON::required(jsWorkerReq, "czarrestart"); - if (czarRestart) { - auto restartCzarId = RequestBodyJSON::required(jsWorkerReq, "czarrestartcancelczid"); - auto restartQueryId = RequestBodyJSON::required(jsWorkerReq, "czarrestartcancelqid"); - wqsData->setCzarCancelAfterRestart(restartCzarId, restartQueryId); - } - return wqsData; - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); - } - return nullptr; -} - -void WorkerQueryStatusData::parseLists(nlohmann::json const& jsWR, TIMEPOINT updateTm) { - lock_guard mapLg(mapMtx); - parseListsInto(jsWR, updateTm, qIdDoneKeepFiles, qIdDoneDeleteFiles, qIdDeadUberJobs); -} - -void WorkerQueryStatusData::parseListsInto(nlohmann::json const& jsWR, TIMEPOINT updateTm, - std::map& doneKeepF, - std::map& doneDeleteF, - std::map>& deadUberJobs) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& a"); - auto& jsQIdDoneKeepFiles = jsWR["qiddonekeepfiles"]; - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& b"); - for (auto const& qidKeep : jsQIdDoneKeepFiles) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& b1"); - doneKeepF[qidKeep] = updateTm; - } - - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& c"); - auto& jsQIdDoneDeleteFiles = jsWR["qiddonedeletefiles"]; - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& d"); - for (auto const& qidDelete : jsQIdDoneDeleteFiles) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& d1"); - doneDeleteF[qidDelete] = updateTm; - } - - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& e"); - auto& jsQIdDeadUberJobs = jsWR["qiddeaduberjobs"]; - LOGS(_log, LOG_LVL_ERROR, - "WorkerQueryStatusData::parseListsInto &&& f jsQIdDeadUberJobs=" << jsQIdDeadUberJobs); - // Interestingly, !jsQIdDeadUberJobs.empty() doesn't work, but .size() > 0 does. - // Not having the size() check causes issues with the for loop trying to read the - // first element of an empty list, which goes badly. - if (jsQIdDeadUberJobs.size() > 0) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& f1"); - for (auto const& qDeadUjs : jsQIdDeadUberJobs) { - LOGS(_log, LOG_LVL_ERROR, "WorkerQueryStatusData::parseListsInto &&& f1a qDeadUjs=" << qDeadUjs); - QueryId qId = qDeadUjs["qid"]; - auto const& ujIds = qDeadUjs["ujids"]; - auto& mapOfUj = deadUberJobs[qId]; - for (auto const& ujId : ujIds) { - LOGS(_log, LOG_LVL_ERROR, - "WorkerQueryStatusData::parseListsInto &&& f1d1 qId=" << qId << " ujId=" << ujId); - mapOfUj[ujId] = updateTm; - } - } - } -} - -void WorkerQueryStatusData::addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm) { - lock_guard mapLg(mapMtx); - auto& ujMap = qIdDeadUberJobs[qId]; - for (auto const ujId : ujIds) { - ujMap[ujId] = tm; - } -} - -void WorkerQueryStatusData::addDeadUberJob(QueryId qId, UberJobId ujId, TIMEPOINT tm) { - lock_guard mapLg(mapMtx); - auto& ujMap = qIdDeadUberJobs[qId]; - ujMap[ujId] = tm; -} - -void WorkerQueryStatusData::addToDoneDeleteFiles(QueryId qId) { - lock_guard mapLg(mapMtx); - qIdDoneDeleteFiles[qId] = CLOCK::now(); -} - -void WorkerQueryStatusData::addToDoneKeepFiles(QueryId qId) { - lock_guard mapLg(mapMtx); - qIdDoneKeepFiles[qId] = CLOCK::now(); -} - -void WorkerQueryStatusData::removeDeadUberJobsFor(QueryId qId) { - lock_guard mapLg(mapMtx); - qIdDeadUberJobs.erase(qId); -} - -json WorkerQueryStatusData::serializeResponseJson(uint64_t workerStartupTime) { - // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a - // response. Nothing should be deleted and time is irrelevant for this, so maxLifetime is enormous - // and any time could be used for last contact, but now() is easy. - // This is only called by the worker. As such nothing should be deleted here as the lifetime of - // these elements is determined by the lifetime of the owning UserQueryInfo instance. - double maxLifetime = std::numeric_limits::max(); - auto now = CLOCK::now(); - json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; - jsResp["w-startup-time"] = workerStartupTime; - addListsToJson(jsResp, now, maxLifetime); - return jsResp; -} - -std::pair WorkerQueryStatusData::handleResponseJson(nlohmann::json const& jsResp) { - auto now = CLOCK::now(); - std::map doneKeepF; - std::map doneDeleteF; - std::map> deadUberJobs; - parseListsInto(jsResp, now, doneKeepF, doneDeleteF, deadUberJobs); - - lock_guard mapLg(mapMtx); - // Remove entries from _qIdDoneKeepFiles - for (auto const& [qId, tm] : doneKeepF) { - qIdDoneKeepFiles.erase(qId); - } - - // Remove entries from _qIdDoneDeleteFiles - for (auto const& [qId, tm] : doneDeleteF) { - qIdDoneDeleteFiles.erase(qId); - } - - // Remove entries from _qIdDeadUberJobs - for (auto const& [qId, ujMap] : deadUberJobs) { - auto iter = qIdDeadUberJobs.find(qId); - if (iter != qIdDeadUberJobs.end()) { - auto& deadMap = iter->second; - for (auto const& [ujId, tm] : ujMap) { - deadMap.erase(ujId); - } - if (deadMap.empty()) { - qIdDeadUberJobs.erase(iter); - } - } - } - - bool workerRestarted = false; - auto workerStartupTime = RequestBodyJSON::required(jsResp, "w-startup-time"); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " workerStartupTime=" << workerStartupTime); - if (!_wInfo->checkWStartupTime(workerStartupTime)) { - LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " startup time for worker=" << _wInfo->dump() - << " changed to=" << workerStartupTime << " Assuming worker restarted"); - workerRestarted = true; - } - return {true, workerRestarted}; -} - -string WorkerQueryStatusData::dump() const { - lock_guard lgI(_infoMtx); - return _dump(); -} - -string WorkerQueryStatusData::_dump() const { - stringstream os; - os << "ActiveWorker " << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); - return os.str(); -} - -} // namespace lsst::qserv::http diff --git a/src/http/WorkerQueryStatusData.h b/src/http/WorkerQueryStatusData.h deleted file mode 100644 index c56c148b0d..0000000000 --- a/src/http/WorkerQueryStatusData.h +++ /dev/null @@ -1,371 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_HTTP_WORKERQUERYSTATUSDATA_H -#define LSST_QSERV_HTTP_WORKERQUERYSTATUSDATA_H - -// System headers -#include -#include -#include -#include -#include - -// Third party headers -#include "nlohmann/json.hpp" - -// qserv headers -#include "global/clock_defs.h" -#include "global/intTypes.h" - -// This header declarations -namespace lsst::qserv::http { - -/// This class just contains the czar id and network contact information. -class CzarContactInfo { -public: - using Ptr = std::shared_ptr; - std::string cName(const char* fnc) const { return std::string("CzarContactInfo") + fnc; } - - CzarContactInfo() = delete; - CzarContactInfo(CzarContactInfo const&) = default; - CzarContactInfo& operator=(CzarContactInfo const&) = default; - - /// &&& doc - bool compare(CzarContactInfo const& other) { - return (czName == other.czName && czId == other.czId && czPort == other.czPort && - czHostName == other.czHostName); - } - - static Ptr create(std::string const& czName_, CzarIdType czId_, int czPort_, - std::string const& czHostName_, uint64_t czStartupTime_) { - return Ptr(new CzarContactInfo(czName_, czId_, czPort_, czHostName_, czStartupTime_)); - } - - static Ptr createFromJson(nlohmann::json const& czarJson); - - std::string const czName; ///< czar "name" - CzarIdType const czId; ///< czar "id" - int const czPort; ///< czar "management-port" - std::string const czHostName; ///< czar "management-host-name" - uint64_t const czStartupTime; ///< czar startup time - - /// &&& doc - nlohmann::json serializeJson() const; - - std::string dump() const; - -private: - CzarContactInfo(std::string const& czName_, CzarIdType czId_, int czPort_, std::string const& czHostName_, - uint64_t czStartupTime_) - : czName(czName_), - czId(czId_), - czPort(czPort_), - czHostName(czHostName_), - czStartupTime(czStartupTime_) {} -}; - -/// This class just contains the worker id and network communication information. -class WorkerContactInfo { -public: - using Ptr = std::shared_ptr; - - using WCMap = std::unordered_map; - using WCMapPtr = std::shared_ptr; - - static Ptr create(std::string const& wId_, std::string const& wHost_, std::string const& wManagementHost_, - int wPort_, TIMEPOINT updateTime_) { - return Ptr(new WorkerContactInfo(wId_, wHost_, wManagementHost_, wPort_, updateTime_)); - } - - /// &&& doc Used to create WorkerQueryStatusData object from a registry json message. - static Ptr createFromJsonRegistry(std::string const& wId_, nlohmann::json const& regJson); - - /// &&& doc Used to create WorkerQueryStatusData object from a worker json message. - static Ptr createFromJsonWorker(nlohmann::json const& workerJson, TIMEPOINT updateTime); - - /// &&& doc - nlohmann::json serializeJson() const; - - std::string cName(const char* fn) { return std::string("WorkerContactInfo::") + fn; } - - std::string const wId; ///< key, this is the one thing that cannot change. - - std::string getWHost() const { - std::lock_guard lg(_rMtx); - return _wHost; - } - - std::string getWManagementHost() const { - std::lock_guard lg(_rMtx); - return _wManagementHost; - } - - int getWPort() const { - std::lock_guard lg(_rMtx); - return _wPort; - } - - /// &&doc - void changeBaseInfo(WorkerContactInfo const& other) { - auto [oWId, oWHost, oWManagementHost, oWPort] = other.getAll(); - std::lock_guard lg(_rMtx); - _wHost = oWHost; - _wManagementHost = oWManagementHost; - _wPort = oWPort; - } - - /// @return wId - workerId - /// @return _wHost - worker host - /// @return _wManagementHost - management host - /// @return _wPort - worker port - std::tuple getAll() const { - std::lock_guard lg(_rMtx); - return {wId, _wHost, _wManagementHost, _wPort}; - } - - /// Return true if communication related items are the same. - bool isSameContactInfo(WorkerContactInfo const& other) const { - auto [oWId, oWHost, oWManagementHost, oWPort] = other.getAll(); - std::lock_guard lg(_rMtx); - return (wId == oWId && _wHost == oWHost && _wManagementHost == oWManagementHost && _wPort == oWPort); - } - - void regUpdateTime(TIMEPOINT updateTime) { - std::lock_guard lg(_rMtx); - _regUpdate = updateTime; - } - - double timeSinceRegUpdateSeconds() const { - std::lock_guard lg(_rMtx); - double secs = std::chrono::duration(CLOCK::now() - _regUpdate).count(); - return secs; - } - - TIMEPOINT getRegUpdate() const { - std::lock_guard lg(_rMtx); - return _regUpdate; - } - - /* &&& - /// Sets _wStartupTime to startupTime, but only if _wStartupTime was 0. - /// @returns true if _wStartupTime was set. - bool setWStartupTime(uint64_t startupTime) { //&&& del if not used - std::lock_guard lg(_rMtx); - if (_wStartupTime == 0) { - _wStartupTime = startupTime; - return true; - } - return false; - } - */ - - /// @return true if startupTime equals _wStartupTime or _wStartupTime was never set, - /// if _wStartupTime was never set, it is set to startupTime. - /// @return false indicates the worker was restarted and all associated jobs need - /// re-assignment. - bool checkWStartupTime(uint64_t startupTime) { - std::lock_guard lg(_rMtx); - if (_wStartupTime == startupTime) { - return true; - } - if (_wStartupTime == 0) { - _wStartupTime = startupTime; - return true; - } - _wStartupTime = startupTime; - return false; - } - - uint64_t getWStartupTime() const { - std::lock_guard lg(_rMtx); - return _wStartupTime; - } - - std::string dump() const; - -private: - WorkerContactInfo(std::string const& wId_, std::string const& wHost_, std::string const& wManagementHost_, - int wPort_, TIMEPOINT updateTime_) - : wId(wId_), _wHost(wHost_), _wManagementHost(wManagementHost_), _wPort(wPort_) { - regUpdateTime(updateTime_); - } - - // _rMtx must be locked before calling - std::string _dump() const; - - // _rMtx must be locked before calling - nlohmann::json _serializeJson() const; - - std::string _wHost; ///< "host-addr" entry. - std::string _wManagementHost; ///< "management-host-name" entry. - int _wPort; ///< "management-port" entry. - - /// Last time the registry heard from this worker. The ActiveWorker class - /// will use this to determine the worker's state. - /// &&& Store in seconds since epoch to make atomic? - TIMEPOINT _regUpdate; - - /// "w-startup-time", it's value is set to zero until the real value is - /// received from the worker. Once it is non-zero, any change indicates - /// the worker was restarted and all UberJobs that were assigned there - /// need to be unassigned. On the worker, this should always be set from - /// foreman()->getStartupTime(); - uint64_t _wStartupTime = 0; - - mutable std::mutex _rMtx; ///< protects _regUpdate -}; - -/// This classes purpose is to be a structure to store and transfer information -/// about which queries have been completed or cancelled on the worker. This -/// class contains the functions that encode and decode the data they contain -/// to and from a json format. -class WorkerQueryStatusData { -public: - using Ptr = std::shared_ptr; - - WorkerQueryStatusData() = delete; - WorkerQueryStatusData(WorkerQueryStatusData const&) = delete; - WorkerQueryStatusData& operator=(WorkerQueryStatusData const&) = delete; - - std::string cName(const char* fName) { return std::string("WorkerQueryStatusData::") + fName; } - - static Ptr create(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_, - std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { - return Ptr(new WorkerQueryStatusData(wInfo_, czInfo_, replicationInstanceId_, replicationAuthKey_)); - } - - /// &&& doc Used to create WorkerQueryStatusData object from a worker json message. - static Ptr createFromJson(nlohmann::json const& czarJson, std::string const& replicationInstanceId_, - std::string const& replicationAuthKey_, TIMEPOINT updateTm); - - ~WorkerQueryStatusData() = default; - - void setWInfo(WorkerContactInfo::Ptr const& wInfo_) { - std::lock_guard lgI(_infoMtx); - if (_wInfo == nullptr) { - _wInfo = wInfo_; - return; - } - if (wInfo_ != nullptr) { - // This only change host and port values of _wInfo. - _wInfo->changeBaseInfo(*wInfo_); - } - } - - WorkerContactInfo::Ptr getWInfo() const { - std::lock_guard lgI(_infoMtx); - return _wInfo; - } - CzarContactInfo::Ptr getCzInfo() const { return _czInfo; } - - /// doc &&& - void addDeadUberJob(QueryId qId, UberJobId ujId, TIMEPOINT tm); - - /// &&& doc - void addDeadUberJobs(QueryId qId, std::vector ujIds, TIMEPOINT tm); - - /// &&& doc - void addToDoneDeleteFiles(QueryId qId); - - /// &&& doc - void addToDoneKeepFiles(QueryId qId); - - /// &&& doc - void removeDeadUberJobsFor(QueryId qId); - - void setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { - std::lock_guard mapLg(mapMtx); - czarCancelAfterRestart = true; - czarCancelAfterRestartCzId = czId; - czarCancelAfterRestartQId = lastQId; - } - - bool isCzarRestart() const { return czarCancelAfterRestart; } - CzarIdType getCzarRestartCzarId() const { return czarCancelAfterRestartCzId; } - QueryId getCzarRestartQueryId() const { return czarCancelAfterRestartQId; } - - /// Create a json object held by a shared pointer to use as a message. - /// Old objects in this instance will be removed after being added to the - /// json message. - std::shared_ptr serializeJson(double maxLifetime); - - /// Add contents of qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs to `jsWR`, - /// and remove map elements that have an age (tmMark - element.touchTime) greater - /// than maxLifetime. - void addListsToJson(nlohmann::json& jsWR, TIMEPOINT tmMark, double maxLifetime); - - /// &&& doc - /// @throws std::invalid_argument - void parseLists(nlohmann::json const& jsWR, TIMEPOINT updateTm); - - /// &&& doc - //&&&nlohmann::json serializeResponseJson(); - nlohmann::json serializeResponseJson(uint64_t workerStartupTime); - - /// &&& doc - //&&&bool handleResponseJson(nlohmann::json const& jsResp); - std::pair handleResponseJson(nlohmann::json const& jsResp); - - /// &&& doc - ///&&&void handleCzarRestart(); - - /// &&& doc - static void parseListsInto(nlohmann::json const& jsWR, TIMEPOINT updateTm, - std::map& doneKeepF, - std::map& doneDeleteF, - std::map>& deadUberJobs); - - std::string dump() const; - - // Making these private requires member functions to be written - // that cause issues with linking. All of the workarounds are ugly. - std::map qIdDoneKeepFiles; ///< &&& doc - limit reached - std::map qIdDoneDeleteFiles; ///< &&& doc -cancelled/finished - std::map> qIdDeadUberJobs; ///< &&& doc - std::atomic czarCancelAfterRestart = false; - CzarIdType czarCancelAfterRestartCzId = 0; - QueryId czarCancelAfterRestartQId = 0; - /// Protects _qIdDoneKeepFiles, _qIdDoneDeleteFiles, _qIdDeadUberJobs, - /// and czarCancelAfter variables. - mutable std::mutex mapMtx; - -private: - WorkerQueryStatusData(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_, - std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) - : _wInfo(wInfo_), - _czInfo(czInfo_), - _replicationInstanceId(replicationInstanceId_), - _replicationAuthKey(replicationAuthKey_) {} - - WorkerContactInfo::Ptr _wInfo; ///< &&& doc - CzarContactInfo::Ptr const _czInfo; //< &&& doc - mutable std::mutex _infoMtx; ///< protects wInfo - - std::string const _replicationInstanceId; ///< &&& doc - std::string const _replicationAuthKey; ///< &&& doc - - /// _infoMtx must be locked before calling. - std::string _dump() const; -}; - -} // namespace lsst::qserv::http - -#endif // LSST_QSERV_HTTP_WORKERQUERYSTATUSDATA_H diff --git a/src/http/testStatusData.cc b/src/http/testStatusData.cc deleted file mode 100644 index d9f5377116..0000000000 --- a/src/http/testStatusData.cc +++ /dev/null @@ -1,169 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// System headers -#include -#include -#include -#include -#include - -// Qserv headers -#include "global/clock_defs.h" -#include "http/WorkerQueryStatusData.h" - -// LSST headers -#include "lsst/log/Log.h" - -// Boost unit test header -#define BOOST_TEST_MODULE RequestQuery -#include - -using namespace std; -namespace test = boost::test_tools; -using namespace lsst::qserv::http; - -BOOST_AUTO_TEST_SUITE(Suite) - -BOOST_AUTO_TEST_CASE(CzarContactInfo) { - string const replicationInstanceId = "repliInstId"; - string const replicationAuthKey = "repliIAuthKey"; - - uint64_t cxrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 5s); - uint64_t wkrStartTime = lsst::qserv::millisecSinceEpoch(lsst::qserv::CLOCK::now() - 10s); - - string const czrName("czar_name"); - lsst::qserv::CzarIdType const czrId = 32; - int czrPort = 2022; - string const czrHost("cz_host"); - - //&&&auto czarA = lsst::qserv::http::CzarContactInfo::create(czrName, czrId, czrPort, czrHost); - auto czarA = lsst::qserv::http::CzarContactInfo::create(czrName, czrId, czrPort, czrHost, cxrStartTime); - LOGS_ERROR("&&& a czarA=" << czarA->dump()); - - auto czarAJs = czarA->serializeJson(); - LOGS_ERROR("&&& b czarAJs=" << czarAJs); - - auto czarB = lsst::qserv::http::CzarContactInfo::createFromJson(czarAJs); - LOGS_ERROR("&&& c czarB=" << czarB); - BOOST_REQUIRE(czarA->compare(*czarB)); - - //&&&auto czarC = lsst::qserv::http::CzarContactInfo::create("different", czrId, czrPort, czrHost); - auto czarC = - lsst::qserv::http::CzarContactInfo::create("different", czrId, czrPort, czrHost, cxrStartTime); - BOOST_REQUIRE(!czarA->compare(*czarC)); - - auto start = lsst::qserv::CLOCK::now(); - auto workerA = WorkerContactInfo::create("sd_workerA", "host_w1", "mgmhost_a", 3421, start); - - auto workerB = WorkerContactInfo::create("sd_workerB", "host_w2", "mgmhost_a", 3421, start); - auto workerC = WorkerContactInfo::create("sd_workerC", "host_w3", "mgmhost_b", 3422, start); - - LOGS_ERROR("&&& d workerA=" << workerA->dump()); - - auto jsWorkerA = workerA->serializeJson(); - LOGS_ERROR("&&& e jsWorkerA=" << jsWorkerA); - auto start1Sec = start + 1s; - auto workerA1 = WorkerContactInfo::createFromJsonWorker(jsWorkerA, start1Sec); - LOGS_ERROR("&&& f workerA1=" << workerA1->dump()); - BOOST_REQUIRE(workerA->isSameContactInfo(*workerA1)); - - // WorkerQueryStatusData - auto wqsdA = lsst::qserv::http::WorkerQueryStatusData::create(workerA, czarA, replicationInstanceId, - replicationAuthKey); - LOGS_ERROR("&&& g wqsdA=" << wqsdA->dump()); - - //&&&double timeoutAliveSecs = 100.0; - //&&&double timeoutDeadSecs = 2*timeoutAliveSecs; - double maxLifetime = 300.0; - auto jsDataA = wqsdA->serializeJson(maxLifetime); - LOGS_ERROR("&&& h jsDataA=" << *jsDataA); - - // Check that empty lists work. - auto wqsdA1 = lsst::qserv::http::WorkerQueryStatusData::createFromJson(*jsDataA, replicationInstanceId, - replicationAuthKey, start1Sec); - LOGS_ERROR("&&& i wqsdA1=" << wqsdA1->dump()); - LOGS_ERROR("&&& i wqsdA=" << wqsdA->dump()); - auto jsDataA1 = wqsdA1->serializeJson(maxLifetime); - LOGS_ERROR("&&& i jsDataA1=" << *jsDataA1); - LOGS_ERROR("&&& i jsDataA=" << *jsDataA); - BOOST_REQUIRE(*jsDataA == *jsDataA1); - - vector qIdsDelFiles = {7, 8, 9, 15, 25, 26, 27, 30}; - vector qIdsKeepFiles = {1, 2, 3, 4, 6, 10, 13, 19, 33}; - for (auto const qIdDF : qIdsDelFiles) { - wqsdA->qIdDoneDeleteFiles[qIdDF] = start; - } - - jsDataA = wqsdA->serializeJson(maxLifetime); - LOGS_ERROR("&&& j jsDataA=" << jsDataA); - BOOST_REQUIRE(*jsDataA != *jsDataA1); - - for (auto const qIdKF : qIdsKeepFiles) { - wqsdA->qIdDoneKeepFiles[qIdKF] = start; - } - - wqsdA->addDeadUberJobs(12, {1, 3}, start); - - LOGS_ERROR("&&& i wqsdA=" << wqsdA->dump()); - - jsDataA = wqsdA->serializeJson(maxLifetime); - LOGS_ERROR("&&& j jsDataA=" << *jsDataA); - - auto start5Sec = start + 5s; - auto workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createFromJson( - *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); - auto jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); - BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); - - wqsdA->addDeadUberJobs(12, {34}, start5Sec); - wqsdA->addDeadUberJobs(91, {77}, start5Sec); - wqsdA->addDeadUberJobs(1059, {1, 4, 6, 7, 8, 10, 3, 22, 93}, start5Sec); - - jsDataA = wqsdA->serializeJson(maxLifetime); - LOGS_ERROR("&&& k jsDataA=" << *jsDataA); - BOOST_REQUIRE(*jsDataA != *jsWorkerAFromJson); - - workerAFromJson = lsst::qserv::http::WorkerQueryStatusData::createFromJson( - *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); - jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); - LOGS_ERROR("&&& l jsWorkerAFromJson=" << *jsWorkerAFromJson); - BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); - - // Make the response, which contains lists of the items handled by the workers. - auto jsWorkerResp = workerAFromJson->serializeResponseJson(wkrStartTime); - - // test removal of elements after response. - BOOST_REQUIRE(!wqsdA->qIdDoneDeleteFiles.empty()); - BOOST_REQUIRE(!wqsdA->qIdDoneKeepFiles.empty()); - BOOST_REQUIRE(!wqsdA->qIdDeadUberJobs.empty()); - - wqsdA->handleResponseJson(jsWorkerResp); - auto [respSuccess, workerRestarted] = wqsdA->handleResponseJson(jsWorkerResp); - BOOST_REQUIRE(respSuccess == true); - BOOST_REQUIRE(workerRestarted == false); - - BOOST_REQUIRE(wqsdA->qIdDoneDeleteFiles.empty()); - BOOST_REQUIRE(wqsdA->qIdDoneKeepFiles.empty()); - BOOST_REQUIRE(wqsdA->qIdDeadUberJobs.empty()); -} - -BOOST_AUTO_TEST_SUITE_END() diff --git a/src/protojson/ScanTableInfo.h b/src/protojson/ScanTableInfo.h index 6f31d9f676..061ea0c0f9 100644 --- a/src/protojson/ScanTableInfo.h +++ b/src/protojson/ScanTableInfo.h @@ -36,7 +36,7 @@ namespace lsst::qserv::protojson { /// Structure to store shared scan information for a single table. /// -struct ScanTableInfo { // &&& check if still useful +struct ScanTableInfo { using ListOf = std::vector; ScanTableInfo() = default; diff --git a/src/protojson/UberJobMsg.cc b/src/protojson/UberJobMsg.cc index e95560de09..312f138a6e 100644 --- a/src/protojson/UberJobMsg.cc +++ b/src/protojson/UberJobMsg.cc @@ -448,6 +448,24 @@ string JobFragment::dump() const { return os.str(); } +string JobFragment::dump() const { + stringstream os; + os << " templateIndexes={"; + for (int j : _jobSubQueryTempIndexes) { + os << j << ", "; + } + os << "} subchunkIds={"; + for (int j : _subchunkIds) { + os << j << ", "; + } + os << "} dbtbl={"; + for (int j : _subchunkIds) { + os << j << ", "; + } + os << "}"; + return os.str(); +} + nlohmann::json JobFragment::serializeJson() const { json jsFragment = {{"subquerytemplate_indexes", _jobSubQueryTempIndexes}, {"dbtables_indexes", _jobDbTablesIndexes}, diff --git a/src/protojson/UberJobMsg.h b/src/protojson/UberJobMsg.h index 1962c7feb7..c4969bcfee 100644 --- a/src/protojson/UberJobMsg.h +++ b/src/protojson/UberJobMsg.h @@ -205,7 +205,6 @@ class JobMsg { int getAttemptCount() const { return _attemptCount; } std::string getChunkQuerySpecDb() const { return _chunkQuerySpecDb; } int getChunkId() const { return _chunkId; } - std::vector const& getChunkScanTableIndexes() const { return _chunkScanTableIndexes; } JobFragment::VectPtr getJobFragments() const { return _jobFragments; } diff --git a/src/protojson/testUberJobMsg.cc b/src/protojson/testUberJobMsg.cc index 95f8d8627f..baf480feaa 100644 --- a/src/protojson/testUberJobMsg.cc +++ b/src/protojson/testUberJobMsg.cc @@ -59,6 +59,49 @@ string testB() { return tb; } +bool parseSerializeReparseCheck(string const& jsStr, string const& note) { + string fName("parseSerialize "); + fName += note + " "; + LOGS(_log, LOG_LVL_INFO, fName << " start " << jsStr); + nlohmann::json js = nlohmann::json::parse(jsStr); + LOGS(_log, LOG_LVL_INFO, fName << " parse 1"); + + UberJobMsg::Ptr ujm = UberJobMsg::createFromJson(js); + BOOST_REQUIRE(ujm != nullptr); + + nlohmann::json jsUjm = ujm->serializeJson(); + LOGS(_log, LOG_LVL_INFO, fName << " serialized jsUjm=" << jsUjm); + + UberJobMsg::Ptr ujmCreated = UberJobMsg::createFromJson(jsUjm); + LOGS(_log, LOG_LVL_INFO, fName << " created"); + nlohmann::json jsUjmCreated = ujmCreated->serializeJson(); + LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); + + bool createdMatchesOriginal = jsUjm == jsUjmCreated; + if (createdMatchesOriginal) { + LOGS(_log, LOG_LVL_INFO, fName << "created matches original"); + } else { + LOGS(_log, LOG_LVL_ERROR, "jsUjm != jsUjmCreated"); + LOGS(_log, LOG_LVL_ERROR, "jsUjm=" << jsUjm); + LOGS(_log, LOG_LVL_ERROR, "jsUjmCreated=" << jsUjmCreated); + } + BOOST_REQUIRE(createdMatchesOriginal); + return createdMatchesOriginal; +} +#endif // &&& + +string testA() { + string ta = + R"({"maxtablesizemb":5432,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; + return ta; +} + +string testB() { + string tb = + R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":39,"worker":"db04"})"; + return tb; +} + bool parseSerializeReparseCheck(string const& jsStr, string const& note) { string fName("parseSerialize "); fName += note + " "; diff --git a/src/qana/QueryMapping.h b/src/qana/QueryMapping.h index 2e8dca319b..585971f976 100644 --- a/src/qana/QueryMapping.h +++ b/src/qana/QueryMapping.h @@ -92,6 +92,8 @@ class QueryMapping { bool hasParameter(Parameter p) const; DbTableSet const& getSubChunkTables() const { return _subChunkTables; } + std::string dump() const { return std::string("&&& NEED CODE"); } + private: ParameterMap _subs; DbTableSet _subChunkTables; diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index 498a0fcf0c..f15024d7a2 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -5,7 +5,6 @@ target_sources(qdisp PRIVATE ChunkMeta.cc CzarStats.cc Executive.cc - JobBase.cc JobDescription.cc JobQuery.cc UberJob.cc diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index c2daea541e..9fd46e807a 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -280,17 +280,24 @@ void Executive::addAndQueueUberJob(shared_ptr const& uj) { } } -void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { // &&& put file collect in the pool ??? +void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { if (_scanInteractive) { _qdispPool->queCmd(cmd, 3); } else { - _qdispPool->queCmd(cmd, 4); + _qdispPool->queCmd(cmd, 3); } } -void Executive::runUberJob(std::shared_ptr const& uberJob) { +void Executive::addAndQueueUberJob(shared_ptr const& uj) { + { + lock_guard lck(_uberJobsMapMtx); + UberJobId ujId = uj->getJobId(); + _uberJobsMap[ujId] = uj; + //&&&uj->setAdded(); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); + } - auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; + auto runUberJobFunc = [uj](util::CmdData*) { uj->runUberJob(); }; auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(runUberJobFunc)); _jobStartCmdList.push_back(cmd); @@ -379,27 +386,7 @@ void Executive::addUberJobs(std::vector> const& uJobsTo for (auto const& uJob : uJobsToAdd) { UberJobId ujId = uJob->getJobId(); _uberJobsMap[ujId] = uJob; - } -} - -void Executive::killIncompleteUberJobsOn(std::string const& restartedWorkerId) { - // Work with a copy to reduce lock time. - std::map> ujobsMap; - { - lock_guard lck(_uberJobsMapMtx); - ujobsMap = _uberJobsMap; - } - for (auto&& [ujKey, uj] : ujobsMap) { - if (uj == nullptr) continue; - auto wContactInfo = uj->getWorkerContactInfo(); - if (wContactInfo->wId == restartedWorkerId) { - if (uj->getStatus()->getState() != qmeta::JobStatus::COMPLETE) { - // All jobs in the uberjob will be set as unassigned, which - // will lead to Czar::_monitor() reassigning them to new - // UberJobs. (Unless this query was cancelled.) - uj->killUberJob(); - } - } + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uJob->getJobCount()); } } @@ -652,9 +639,41 @@ void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { } void Executive::sendWorkersEndMsg(bool deleteResults) { - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " terminating this query deleteResults=" - << deleteResults); - czar::Czar::getCzar()->getCzarRegistry()->endUserQueryOnWorkers(_id, deleteResults); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " terminating this query deleteResults=" << deleteResults); + auto cz = czar::Czar::getCzar(); + if (cz != nullptr) { // Possible in unit tests. + cz->getCzarRegistry()->endUserQueryOnWorkers(_id, deleteResults); + } +} + +void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { + if (_cancelled) { + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " irrelevant as query already cancelled"); + return; + } + + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " killing incomplete UberJobs on " << workerId); + deque ujToCancel; + { + lock_guard lockUJMap(_uberJobsMapMtx); + for (auto const& [ujKey, ujPtr] : _uberJobsMap) { + auto ujStatus = ujPtr->getStatus()->getState(); + if (ujStatus != qmeta::JobStatus::RESPONSE_DONE && ujStatus != qmeta::JobStatus::COMPLETE) { + // RESPONSE_DONE indicates the result file has been read by + // the czar, so before that point the worker's data is + // likely destroyed. COMPLETE indicates all jobs in the + // UberJob are complete. + if (ujPtr->getWorkerContactInfo()->wId == workerId) { + ujToCancel.push_back(ujPtr); + } + } + } + } + + for (auto const& uj : ujToCancel) { + uj->killUberJob(); + uj->setStatusIfOk(qmeta::JobStatus::CANCEL, getIdStr() + " killIncomplete on worker=" + workerId); + } } int Executive::getNumInflight() const { diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 732c0d70d3..583707c7a6 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -118,7 +118,9 @@ class Executive : public std::enable_shared_from_this { /// Find the UberJob with `ujId`. std::shared_ptr findUberJob(UberJobId ujId); - std::string cName(const char* funcName = "") { return std::string("Executive::") + funcName; } + std::string cName(const char* funcName = "") { + return std::string("Executive::") + funcName + " " + getIdStr(); + } /// Set the UserQuerySelect object for this query so this Executive can ask it to make new /// UberJobs in the future, if needed. @@ -140,7 +142,7 @@ class Executive : public std::enable_shared_from_this { void queueFileCollect(std::shared_ptr const& cmd); /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. - void queueFileCollect(std::shared_ptr const& cmd); // &&& delete ??? + void queueFileCollect(std::shared_ptr const& cmd); /// Waits for all jobs on _jobStartCmdList to start. This should not be called /// before ALL jobs have been added to the pool. diff --git a/src/qdisp/JobBase.cc b/src/qdisp/JobBase.cc deleted file mode 100644 index a5ef5a8c8f..0000000000 --- a/src/qdisp/JobBase.cc +++ /dev/null @@ -1,54 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "qdisp/JobBase.h" - -// System headers -#include - -// Qserv headers - -// LSST headers -#include "lsst/log/Log.h" - -using namespace std; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobBase"); -} - -namespace lsst { namespace qserv { namespace qdisp { - -std::ostream& JobBase::dumpOS(std::ostream& os) const { - os << "JobBase no data members"; - return os; -} - -std::string JobBase::dump() const { - std::ostringstream os; - dumpOS(os); - return os.str(); -} - -std::ostream& operator<<(std::ostream& os, JobBase const& jb) { return jb.dumpOS(os); } - -}}} // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h deleted file mode 100644 index b6b18d3252..0000000000 --- a/src/qdisp/JobBase.h +++ /dev/null @@ -1,79 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_QDISP_JOBBASE_H -#define LSST_QSERV_QDISP_JOBBASE_H - -// System headers -#include -#include - -// Qserv headers -#include "global/intTypes.h" - -namespace lsst::qserv::qmeta { -class JobStatus; -} - -// This header declarations -namespace lsst::qserv::qdisp { - -class Executive; -class QdispPool; -class ResponseHandler; -class QueryRequest; - -/// Base class for JobQuery and UberJob. -/// TODO:UJ This could use a lot of cleanup. Once UberJobs are fully in effect, there's no need -/// for this base class as it won't be possible to send a JobQuery to a worker without -/// putting it in an UberJob first. The UberJob is a wrapper that stores worker contact -/// info. -// &&& delete this class as JobQuery and UberJob should no longer have much in common -class JobBase : public std::enable_shared_from_this { -public: - using Ptr = std::shared_ptr; - - JobBase() = default; - JobBase(JobBase const&) = delete; - JobBase& operator=(JobBase const&) = delete; - virtual ~JobBase() = default; - - virtual QueryId getQueryId() const = 0; - virtual UberJobId getJobId() const = 0; - virtual std::string const& getIdStr() const = 0; - //&&&virtual std::shared_ptr getQdispPool() = 0; - //&&& virtual std::string const& getPayload() const = 0; ///< const& in return type is essential for - // xrootd - virtual std::shared_ptr getRespHandler() = 0; - virtual std::shared_ptr getStatus() = 0; - virtual bool getScanInteractive() const = 0; - virtual bool isQueryCancelled() = 0; - virtual void callMarkCompleteFunc(bool success) = 0; - virtual std::shared_ptr getExecutive() = 0; - - virtual std::ostream& dumpOS(std::ostream& os) const; - - std::string dump() const; - friend std::ostream& operator<<(std::ostream& os, JobBase const& jb); -}; - -} // namespace lsst::qserv::qdisp - -#endif // LSST_QSERV_QDISP_JOBBASE_H diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index c2d61759d4..72b0a059fa 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -92,7 +92,7 @@ class JobDescription { std::shared_ptr getJsForWorker() { return _jsForWorker; } - void resetJsForWorker() { _jsForWorker.reset(); } // TODO:UJ may need mutex for _jsForWorker //&&& + void resetJsForWorker() { _jsForWorker.reset(); } friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 21b8dd8174..bf4246be49 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -44,23 +44,19 @@ using namespace std; namespace lsst::qserv::qdisp { JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - qmeta::JobStatus::Ptr const& jobStatus, - shared_ptr const& markCompleteFunc, QueryId qid) - : JobBase(), - _executive(executive), + qmeta::JobStatus::Ptr const& jobStatus, QueryId qid) + : _executive(executive), _jobDescription(jobDescription), - _markCompleteFunc(markCompleteFunc), _jobStatus(jobStatus), _qid(qid), _idStr(QueryIdHelper::makeIdStr(qid, getJobId())) { - //&&&_qdispPool = executive->getQdispPool(); LOGS(_log, LOG_LVL_TRACE, "JobQuery desc=" << _jobDescription); } JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_TRACE, "~JobQuery QID=" << _idStr); } /// Cancel response handling. Return true if this is the first time cancel has been called. -bool JobQuery::cancel(bool superfluous) { /// &&& This can probably be simplified more +bool JobQuery::cancel(bool superfluous) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); if (_cancelled.exchange(true) == false) { LOGS(_log, LOG_LVL_TRACE, "JobQuery::cancel() " << superfluous); diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index f069558673..c4092e34fd 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -33,7 +33,6 @@ // Qserv headers #include "qdisp/Executive.h" -#include "qdisp/JobBase.h" #include "qdisp/JobDescription.h" #include "qdisp/ResponseHandler.h" #include "util/InstanceCount.h" @@ -65,7 +64,7 @@ class JobQuery { JobDescription::Ptr getDescription() { return _jobDescription; } qmeta::JobStatus::Ptr getStatus() { return _jobStatus; } bool cancel(bool superfluous = false); - bool isQueryCancelled() override; + bool isQueryCancelled(); std::shared_ptr getExecutive() { return _executive.lock(); } @@ -123,14 +122,20 @@ class JobQuery { /// @return true if _uberJobId was set, it can only be set if it is unassigned /// or by the current owner. - /// NOTE: _rmutex must be held before calling this + /// NOTE: _jqMtx must be held before calling this bool _setUberJobId(UberJobId ujId); - /// NOTE: _rmutex must be held before calling this - UberJobId _getUberJobId() const { return _uberJobId; } + /// NOTE: _jqMtx must be held before calling this + UberJobId _getUberJobId() const { + VMUTEX_HELD(_jqMtx); + return _uberJobId; + } - /// NOTE: _rmutex must be held before calling this - bool _isInUberJob() const { return _uberJobId >= 0; } + /// NOTE: _jqMtx must be held before calling this + bool _isInUberJob() const { + VMUTEX_HELD(_jqMtx); + return _uberJobId >= 0; + } // Values that don't change once set. std::weak_ptr _executive; diff --git a/src/qdisp/SharedResources.h b/src/qdisp/SharedResources.h deleted file mode 100644 index 6ca6eb8a31..0000000000 --- a/src/qdisp/SharedResources.h +++ /dev/null @@ -1,64 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_SHAREDRESOURCES_H -#define LSST_QSERV_SHAREDRESOURCES_H - -// System headers -#include - -namespace lsst::qserv::util { // &&& delete -class QdispPool; -} - -namespace lsst::qserv::qdisp { - -/// Put resources that all Executives need to share in one class to reduce -/// the number of arguments passed. -/// This class should be kept simple so it can easily be included in headers -/// without undue compiler performances problems. - // &&& there's nothing in here but qdisppool!? Try to delete, but there - // &&& will probably be unit test issues. -class SharedResources { -public: - using Ptr = std::shared_ptr; - - static Ptr create(std::shared_ptr const& qdispPool) { - return Ptr(new SharedResources(qdispPool)); - } - - SharedResources() = delete; - SharedResources(SharedResources const&) = delete; - SharedResources& operator=(SharedResources const&) = delete; - ~SharedResources() = default; - - std::shared_ptr getQdispPool() { return _qdispPool; } //&&& delete - -private: - SharedResources(std::shared_ptr const& qdispPool) : _qdispPool(qdispPool) {} - - /// Thread pool for handling Responses from XrdSsi. - std::shared_ptr _qdispPool; -}; - -} // namespace lsst::qserv::qdisp - -#endif // LSST_QSERV_SHAREDRESOURCES_H diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index bd8ca95b71..7cc95528dc 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -43,6 +43,7 @@ #include "qproc/ChunkQuerySpec.h" #include "util/Bug.h" #include "util/common.h" +#include "util/Histogram.h" //&&& #include "util/QdispPool.h" // LSST headers @@ -100,8 +101,12 @@ bool UberJob::addJob(JobQuery::Ptr const& job) { return success; } -void UberJob::runUberJob() { +util::HistogramRolling histoRunUberJob("&&&uj histoRunUberJob", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); +util::HistogramRolling histoUJSerialize("&&&uj histoUJSerialize", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); + +void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelled LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj start"); // Build the uberjob payload for each job. nlohmann::json uj; unique_lock jobsLock(_jobsMtx); @@ -163,6 +168,10 @@ void UberJob::runUberJob() { LOGS(_log, LOG_LVL_WARN, requestContext + " ujresponse failed, ex: " + ex.what()); exceptionWhat = ex.what(); } + auto endclient = CLOCK::now(); //&&& + std::chrono::duration secsclient = endclient - startclient; // &&& + histoRunUberJob.addEntry(endclient, secsclient.count()); //&&& + LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoRunUberJob.getString("")); if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure, try to send jobs elsewhere"); _unassignJobs(); // locks _jobsMtx @@ -171,6 +180,7 @@ void UberJob::runUberJob() { } else { setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx } + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj runuj end"); return; } @@ -236,15 +246,6 @@ bool UberJob::isQueryCancelled() { return exec->getCancelled(); } -bool UberJob::getScanInteractive() const { - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " _executive == nullptr"); - return false; // Safer to assume the worst. - } - return exec->getScanInteractive(); -} - bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg) { // must be locked _jobsMtx auto currentState = _jobStatus->getState(); @@ -561,7 +562,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return _importResultError(true, "cancelled", "Query cancelled - no executive"); } - if (exec->isLimitRowComplete()) { + if (exec->isRowLimitComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, @@ -578,16 +579,15 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); } - JobBase::Ptr jBaseThis = shared_from_this(); - weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); - + weak_ptr ujThis = weak_from_this(); // TODO:UJ lambda may not be the best way to do this, alsocheck synchronization - may need a mutex for // merging. - auto fileCollectFunc = [ujThis, fileUrl, rowCount](util::CmdData*) { + string const idStr = _idStr; + auto fileCollectFunc = [ujThis, fileUrl, rowCount, idStr](util::CmdData*) { auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { LOGS(_log, LOG_LVL_DEBUG, - "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); return; } uint64_t resultRows = 0; @@ -626,14 +626,14 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { return _workerErrorFinish(deleteData, "cancelled"); } - if (exec->isLimitRowComplete()) { + if (exec->isRowLimitComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } - return _workerErrorFinish(keepData, "none", "limitRowComplete"); + return _workerErrorFinish(keepData, "none", "rowLimitComplete"); } // Currently there are no detectable recoverable errors from workers. The only @@ -691,8 +691,15 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str return jsRet; } -nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { +void UberJob::_importResultFinish(uint64_t resultRows) { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); + + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); + return; + } + /// If this is called, the file has been collected and the worker should delete it /// /// This function should call markComplete for all jobs in the uberjob @@ -700,22 +707,16 @@ nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); if (!statusSet) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " failed to set status " << getIdStr()); - return {{"success", 0}, {"errortype", "statusMismatch"}, {"note", "failed to set status"}}; - } - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); - return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to set status, squashing " << getIdStr()); + // Something has gone very wrong + exec->squash(); + return; } bool const success = true; callMarkCompleteFunc(success); // sets status to COMPLETE exec->addResultRows(resultRows); exec->checkLimitRowComplete(); - - json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; - return jsRet; } nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, @@ -744,7 +745,7 @@ void UberJob::killUberJob() { return; } - if (exec->isLimitRowComplete()) { + if (exec->isRowLimitComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already."); @@ -761,8 +762,7 @@ void UberJob::killUberJob() { _unassignJobs(); // Let Czar::_monitor reassign jobs - other UberJobs are probably being killed - // so waiting probably gets a better distribution. If this is deemed to slow, - // then exec->assignJobsToUberJobs() could be called here. + // so waiting probably gets a better distribution. return; } diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 0a30aabb72..254ee9d7e0 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -181,6 +181,125 @@ class ExecutiveUT : public Executive { } // namespace lsst::qserv::qdisp +namespace lsst::qserv::qdisp { + +class ExecutiveUT; + +class TestInfo : public ResponseHandler { +public: + using Ptr = std::shared_ptr; + + TestInfo() {} + virtual ~TestInfo() {} + + bool goWait() { + unique_lock ulock(_infoMtx); + _infoCV.wait(ulock, [this]() { return _go == true; }); + return _ok; + } + + void setGo(bool val) { + lock_guard lg(_infoMtx); + _go = val; + _infoCV.notify_all(); + } + + // virtual function that won't be needed + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) override { + return {true, false}; + } + void flushHttpError(int errorCode, std::string const& errorMsg, int status) override {} + void errorFlush(std::string const& msg, int code) override {}; + Error getError() const override { return util::Error(); } + void processCancel() override {}; + void prepScrubResults(int jobId, int attempt) override {}; + + /// Print a string representation of the receiver to an ostream + std::ostream& print(std::ostream& os) const override { + os << "TestInfo ujCount=" << ujCount; + return os; + } + + atomic ujCount = 0; + +private: + bool _ok = true; + bool _go = true; + mutex _infoMtx; + condition_variable _infoCV; +}; + +/// Version of UberJob specifically for this unit test. +class UberJobUT : public UberJob { +public: + using PtrUT = std::shared_ptr; + + UberJobUT(std::shared_ptr const& executive, + std::shared_ptr const& respHandler, int queryId, int uberJobId, + qmeta::CzarId czarId, int rowLimit, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData, + TestInfo::Ptr const& testInfo_) + : UberJob(executive, respHandler, queryId, uberJobId, czarId, rowLimit, workerData), + testInfo(testInfo_) {} + + void runUberJob() override { + LOGS(_log, LOG_LVL_INFO, "runUberJob() chunkId=" << chunkId); + bool ok = testInfo->goWait(); + int c = -1; + if (ok) { + c = ++(testInfo->ujCount); + } + callMarkCompleteFunc(ok); + LOGS(_log, LOG_LVL_INFO, "runUberJob() end chunkId=" << chunkId << " c=" << c); + } + + TestInfo::Ptr testInfo; + int chunkId = -1; +}; + +/// Version of Executive specifically for this unit test. +class ExecutiveUT : public Executive { +public: + using PtrUT = shared_ptr; + + ~ExecutiveUT() override = default; + + ExecutiveUT(ExecutiveConfig const& cfg, shared_ptr const& ms, + util::QdispPool::Ptr const& qdispPool, shared_ptr const& qStatus, + shared_ptr const& querySession, TestInfo::Ptr const& testInfo_) + : Executive(cfg, ms, qdispPool, qStatus, querySession), testInfo(testInfo_) {} + + void assignJobsToUberJobs() override { + vector ujVect; + + // Make an UberJobUnitTest for each job + qdisp::Executive::ChunkIdJobMapType unassignedChunks = unassignedChunksInQuery(); + for (auto const& [chunkId, jqPtr] : unassignedChunks) { + auto exec = shared_from_this(); + PtrUT execUT = dynamic_pointer_cast(exec); + auto uJob = UberJobUT::PtrUT(new UberJobUT(execUT, testInfo, getId(), ujId++, czarId, rowLimit, + targetWorker, testInfo)); + uJob->chunkId = chunkId; + uJob->addJob(jqPtr); + ujVect.push_back(uJob); + } + + for (auto const& ujPtr : ujVect) { + addAndQueueUberJob(ujPtr); + } + LOGS(_log, LOG_LVL_INFO, "assignJobsToUberJobs() end"); + } + + CzarIdType czarId = 1; + UberJobId ujId = 1; + int rowLimit = 0; + czar::CzarChunkMap::WorkerChunksData::Ptr targetWorker = nullptr; + + TestInfo::Ptr testInfo; +}; + +} // namespace lsst::qserv::qdisp + qdisp::JobDescription::Ptr makeMockJobDescription(qdisp::Executive::Ptr const& ex, int sequence, ResourceUnit const& ru, std::string msg, std::shared_ptr const& mHandler) { diff --git a/src/qproc/ChunkQuerySpec.h b/src/qproc/ChunkQuerySpec.h index ef0d8325c4..f9924e9bc8 100644 --- a/src/qproc/ChunkQuerySpec.h +++ b/src/qproc/ChunkQuerySpec.h @@ -65,7 +65,8 @@ class ChunkQuerySpec { bool scanInteractive{false}; DbTableSet subChunkTables; std::vector subChunkIds; - std::vector queries; + std::vector queries; // &&& remove if possible + std::vector queryTemplates; // Consider promoting the concept of container of ChunkQuerySpec // in the hopes of increased code cleanliness. std::shared_ptr nextFragment; ///< ad-hoc linked list (consider removal) diff --git a/src/qproc/QuerySession.cc b/src/qproc/QuerySession.cc index 969409a4dc..9bd643265b 100644 --- a/src/qproc/QuerySession.cc +++ b/src/qproc/QuerySession.cc @@ -391,6 +391,7 @@ std::vector QuerySession::_buildChunkQueries(query::QueryTemplate:: } for (auto&& queryTemplate : queryTemplates) { + LOGS(_log, LOG_LVL_WARN, "&&&uj QuerySession::_buildChunkQueries qt=" << queryTemplate.dump()); std::string str = _context->queryMapping->apply(chunkSpec, queryTemplate); chunkQueries.push_back(std::move(str)); } diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h deleted file mode 100644 index 1e3bfd3bef..0000000000 --- a/src/qproc/TaskMsgFactory.h +++ /dev/null @@ -1,75 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2017 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_QPROC_TASKMSGFACTORY_H -#define LSST_QSERV_QPROC_TASKMSGFACTORY_H -/** - * @file - * - * @brief TaskMsgFactory is a factory for TaskMsg (protobuf) objects. - * - * @author Daniel L. Wang, SLAC - */ - -// System headers -#include -#include - -// Third party headers -#include "nlohmann/json.hpp" - -// Qserv headers -#include "global/DbTable.h" -#include "global/intTypes.h" -#include "proto/worker.pb.h" -#include "qmeta/types.h" - -namespace lsst::qserv::qproc { - -class ChunkQuerySpec; - -/// TaskMsgFactory is a factory for TaskMsg (protobuf) objects. -/// All member variables must be thread safe. -/// &&& fix doc -class TaskMsgFactory { -public: - using Ptr = std::shared_ptr; - - TaskMsgFactory() = default; - virtual ~TaskMsgFactory() {} - - /// Make and return the json message for a single Job. - virtual std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, - std::string const& chunkResultName, QueryId queryId, - int jobId, int attemptCount, qmeta::CzarId czarId); - -private: - /// Make a json message for a single fragment. - void _addFragmentJson(nlohmann::json& jsFragments, std::string const& resultName, - DbTableSet const& subChunkTables, std::vector const& subChunkIds, - std::vector const& queries); -}; - -} // namespace lsst::qserv::qproc - -#endif // LSST_QSERV_QPROC_TASKMSGFACTORY_H diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index a0c146e65a..ed60a7fe85 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -294,7 +294,7 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response return true; } auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { return true; } diff --git a/src/util/xrootd.cc b/src/util/xrootd.cc deleted file mode 100644 index bde271719a..0000000000 --- a/src/util/xrootd.cc +++ /dev/null @@ -1,91 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2009-2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -// xrootd.h -- Helper funcitons for xrootd-based dispatch - -#include "util/xrootd.h" - -// System headers -#include -#include - -// Third-party headers -#include "boost/format.hpp" - -/// &&& file seems unused, delete if possible - -namespace lsst::qserv::util { - -std::string makeUrl(char const* hostport, char const* typeStr, int chunk) { - std::stringstream s; - s << chunk; - // boost::format version is 5x slower. - // std::string s = (boost::format("%d") % chunk).str(); - return makeUrl(hostport, typeStr, s.str()); -} - -std::string makeUrl(char const* hostport, std::string const& path) { - return makeUrl(hostport, nullptr, path); -} - -std::string makeUrl(char const* hostport, char const* typeStr, std::string const& s, char mode) { - // typeStr is either "query" or "result" - if (!hostport) { - hostport = ::getenv("QSERV_XRD"); - if (!hostport) { - // use local host name if nothing is specified - hostport = "localhost:1094"; - } - } -#if 0 - char* user = "qsmaster"; - boost::format f("xroot://%s@%s//%s/%s"); - return (f % user % hostport % typeStr % s).str(); -#else - // This is ~8.5x faster than the boost::format version. - std::string pfx = "xroot://"; - std::string user("qsmaster"); - std::string tstr; - std::string ret; - if (typeStr) tstr = typeStr; - - if (mode != '\0') { - user += "."; - user += mode; - } - ret.reserve(pfx.size() + user.size() + 1 + 2 + 1 + tstr.size() + s.size()); - ret += pfx; - ret += user; - ret += "@"; - ret += hostport; - ret += "/"; - if (typeStr) { - ret += "/"; - ret += typeStr; - ret += "/"; - } // else: assume s contains leading "/" - ret += s; - return ret; -#endif -} - -} // namespace lsst::qserv::util diff --git a/src/util/xrootd.h b/src/util/xrootd.h deleted file mode 100644 index 947db582bd..0000000000 --- a/src/util/xrootd.h +++ /dev/null @@ -1,44 +0,0 @@ -// -*- LSST-C++ -*- - -/* - * LSST Data Management System - * Copyright 2008, 2009, 2010 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_UTIL_XROOTD_H -#define LSST_QSERV_UTIL_XROOTD_H - -// xrootd.h : consolidates xrootd/lower-level helper functions (i.e., -// dealing with xrootd URLs) - -// Third-party headers -#include - -/// &&& file seems unused, delete if possible - -namespace lsst::qserv::util { - -std::string makeUrl(char const* hostport, char const* typeStr, int chunk); -std::string makeUrl(char const* hostport, char const* typeStr, std::string const& s, char mode = 0); -std::string makeUrl(char const* hostport, std::string const& path); - -} // namespace lsst::qserv::util - -#endif // LSST_QSERV_UTIL_XROOTD_H diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index f0583cfe07..f286ac9760 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -433,6 +433,16 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetIdStr()); } + int const ujRowLimit = task->getRowLimit(); + bool rowLimitComplete = false; + if (ujRowLimit > 0 && _rowcount >= ujRowLimit) { + // There are enough rows to satisfy the query, so stop reading + hasMoreRows = false; + rowLimitComplete = true; + LOGS(_log, LOG_LVL_DEBUG, + __func__ << " enough rows for query rows=" << _rowcount << " " << task->getIdStr()); + } + // If no more rows are left in the task's result set then we need to check // if this is last task in a logical group of ones created for processing // the current request (note that certain classes of requests may require diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 03251465ee..8d231a375a 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -110,7 +110,7 @@ atomic taskSequence{0}; ///< Unique identifier source for Task. /// the util::CommandThreadPool is not called here. Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, size_t templateId, bool hasSubchunks, int subchunkId, string const& db, - vector const& fragSubTables, vector const& fragSubchunkIds, + vector const& fragSubTables, vector const& fragSubchunkIds, shared_ptr const& sc, std::shared_ptr const& queryStats_) : _logLvlWT(LOG_LVL_WARN), @@ -271,128 +271,6 @@ std::vector Task::createTasksFromUberJobMsg( return vect; } -std::vector Task::createTasksForChunk( - std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, proto::ScanInfo const& scanInfo, - bool scanInteractive, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr, mysql::MySqlConfig const& mySqlConfig, - std::shared_ptr const& sqlConnMgr, - std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { - QueryId qId = ujData->getQueryId(); - UberJobId ujId = ujData->getUberJobId(); - CzarIdType czId = ujData->getCzarId(); - - //&&&UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); - wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); - UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); - - string funcN(__func__); - funcN += " QID=" + to_string(qId) + " "; - - vector vect; - for (auto const& job : jsJobs) { - json const& jsJobDesc = job["jobdesc"]; - http::RequestBodyJSON rbJobDesc(jsJobDesc); - // See qproc::TaskMsgFactory::makeMsgJson for message construction. - auto const jdCzarId = rbJobDesc.required("czarId"); - auto const jdQueryId = rbJobDesc.required("queryId"); - if (jdQueryId != qId) { - throw TaskException(ERR_LOC, string("ujId=") + to_string(ujId) + " qId=" + to_string(qId) + - " QueryId mismatch Job qId=" + to_string(jdQueryId)); - } - auto const jdJobId = rbJobDesc.required("jobId"); - auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - auto const jdScanPriority = rbJobDesc.required("scanPriority"); - auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); - auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); - auto const jdChunkId = rbJobDesc.required("chunkId"); - LOGS(_log, LOG_LVL_TRACE, - funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId - << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive - << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); - - auto const jdQueryFragments = rbJobDesc.required("queryFragments"); - int fragmentNumber = 0; - for (auto const& frag : jdQueryFragments) { - vector fragSubQueries; - vector fragSubchunkIds; - vector fragSubTables; - LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); - http::RequestBodyJSON rbFrag(frag); - auto const& jsQueries = rbFrag.required("queries"); - // TODO:UJ move to uberjob???, these should be the same for all jobs - for (auto const& subQ : jsQueries) { - http::RequestBodyJSON rbSubQ(subQ); - auto const subQuery = rbSubQ.required("subQuery"); - LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); - fragSubQueries.push_back(subQuery); - } - auto const& resultTable = rbFrag.required("resultTable"); - auto const& jsSubIds = rbFrag.required("subchunkIds"); - for (auto const& scId : jsSubIds) { - fragSubchunkIds.push_back(scId); - } - auto const& jsSubTables = rbFrag.required("subchunkTables"); - - for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? - http::RequestBodyJSON rbScDbTable(scDbTable); - string scDb = rbScDbTable.required("scDb"); - string scTable = rbScDbTable.required("scTable"); - TaskDbTbl scDbTbl(scDb, scTable); - fragSubTables.push_back(scDbTbl); - } - - for (string const& fragSubQ : fragSubQueries) { - size_t templateId = userQueryInfo->addTemplate(fragSubQ); - if (fragSubchunkIds.empty()) { - bool const noSubchunks = false; - int const subchunkId = -1; - /* &&& - auto task = Task::Ptr(new Task( - ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, userQueryInfo, - templateId, noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, - maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, queryStats, - resultsHttpPort)); - */ - auto task = Task::Ptr(new Task( - ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, templateId, - noSubchunks, subchunkId, jdQuerySpecDb, scanInfo, scanInteractive, maxTableSizeMb, - fragSubTables, fragSubchunkIds, sendChannel, queryStats, resultsHttpPort)); - - vect.push_back(task); - } else { - for (auto subchunkId : fragSubchunkIds) { - bool const hasSubchunks = true; - auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, - fragmentNumber, templateId, hasSubchunks, subchunkId, - jdQuerySpecDb, scanInfo, scanInteractive, - maxTableSizeMb, fragSubTables, fragSubchunkIds, - sendChannel, queryStats, resultsHttpPort)); - /* &&& - auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, - fragmentNumber, userQueryInfo, templateId, - hasSubchunks, subchunkId, jdQuerySpecDb, scanInfo, - scanInteractive, maxTableSizeMb, fragSubTables, - fragSubchunkIds, sendChannel, queryStats, - resultsHttpPort)); - */ - vect.push_back(task); - } - } - } - ++fragmentNumber; - } - } - - for (auto taskPtr : vect) { - // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. - taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, - sqlConnMgr, queriesAndChunks)); - } - return vect; -} std::vector Task::createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, @@ -464,7 +342,6 @@ std::vector Task::createTasksForUnitTest( auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, 0, noSubchunks, subchunkId, jdQuerySpecDb, fragSubTables, fragSubchunkIds, sendChannel, nullptr)); - vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 12cd5c5dcb..ff184d0a93 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -181,6 +181,18 @@ class Task : public util::CommandForThreadPool { std::shared_ptr const& sendChannel, int maxTableSizeMb, std::shared_ptr const& chunkResourceMgr); + //&&& + static std::vector createTasksForUnitTest( + std::shared_ptr const& ujData, nlohmann::json const& jsJobs, + std::shared_ptr const& sendChannel, + protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, + std::shared_ptr const& chunkResourceMgr + //&&&mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& + // sqlConnMgr, + //&&&std::shared_ptr const& queriesAndChunks, + //&&&uint16_t resultsHttpPort = 8080); + ); + std::shared_ptr getSendChannel() const { return _sendChannel; } std::string user; ///< Incoming username // Note that manpage spec of "26 bytes" is insufficient diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 4d6db29e29..6e6b7f67b9 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -113,7 +113,8 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount workerIdStr = _foreman->chunkInventory()->id(); } else { workerIdStr = "dummyWorkerIdStr"; - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " _foreman was null, which should only happen in unit tests"); + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " _foreman was null, which should only happen in unit tests"); } json request = {{"version", http::MetaModule::version}, @@ -350,11 +351,21 @@ string UJTransmitCmd::cName(const char* funcN) const { } void UJTransmitCmd::action(util::CmdData* data) { + // Make certain _selfPtr is reset before leaving this function. + // If a retry is needed, duplicate() is called. + class ResetSelf { + public: + ResetSelf(UJTransmitCmd* ujtCmd) : _ujtCmd(ujtCmd) {} + ~ResetSelf() { _ujtCmd->_selfPtr.reset(); } + UJTransmitCmd* const _ujtCmd; + }; + ResetSelf resetSelf(this); + _attemptCount++; - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&& start attempt=" << _attemptCount); auto ujPtr = _ujData.lock(); if (ujPtr == nullptr || ujPtr->getCancelled()) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " UberJob was cancelled " << _attemptCount); + return; } http::Client client(_method, _url, _requestStr, _headers); bool transmitSuccess = false; @@ -362,17 +373,12 @@ void UJTransmitCmd::action(util::CmdData* data) { json const response = client.readAsJson(); if (0 != response.at("success").get()) { transmitSuccess = true; - _selfPtr.reset(); // clear so this can be deleted. } else { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " Transmit success == 0"); // There's no point in re-sending as the czar got the message and didn't like // it. - // &&& maybe add this czId+ujId to a list of failed uberjobs that can be put - // &&& status return??? Probably overkill. - _selfPtr.reset(); // clear so this can be deleted. } } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&& start d except"); LOGS(_log, LOG_LVL_WARN, cName(__func__) + " " + _requestContext + " failed, ex: " + ex.what()); } @@ -380,10 +386,11 @@ void UJTransmitCmd::action(util::CmdData* data) { auto sPtr = _selfPtr; if (_foreman != nullptr && sPtr != nullptr) { // Do not reset _selfPtr as re-queuing may be needed several times. - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no response for transmit, putting on failed transmit queue."); + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " no response for transmit, putting on failed transmit queue."); auto wCzInfo = _foreman->getWCzarInfoMap()->getWCzarInfo(_czarId); - // This will check if the czar is believed to be alive and try the queue the query to be tried again - // at a lower priority. It it thinks the czar is dead, it will throw it away. + // This will check if the czar is believed to be alive and try the queue the query to be tried + // again at a lower priority. It it thinks the czar is dead, it will throw it away. // TODO:UJ &&& I have my doubts about this as a reconnected czar may go down in flames // &&& as it is hit with thousands of these. // &&& Alternate plan, set a flag in the status message response (WorkerQueryStatusData) @@ -394,25 +401,21 @@ void UJTransmitCmd::action(util::CmdData* data) { auto wPool = _foreman->getWPool(); if (wPool != nullptr) { Ptr replacement = duplicate(); - _selfPtr.reset(); if (replacement != nullptr) { wPool->queCmd(replacement, 2); } else { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " replacement was null"); } - } else{ + } else { // No thread pool, should only be possible in unit tests. LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no wPool"); - _selfPtr.reset(); return; } } } else { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _selfPtr was null, assuming job killed."); - _selfPtr.reset(); // In case _foreman is null. } } - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&& start end"); } void UJTransmitCmd::kill() { @@ -420,8 +423,9 @@ void UJTransmitCmd::kill() { LOGS(_log, LOG_LVL_WARN, funcN); auto sPtr = _selfPtr; _selfPtr.reset(); - if (sPtr == nullptr) { return; } - // &&& TODO:UJ Is there anything that should be done here??? + if (sPtr == nullptr) { + return; + } } UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { @@ -432,7 +436,6 @@ UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { Ptr newPtr = create(_foreman, ujD, _method, _headers, _url, _requestContext, _requestStr); newPtr->_attemptCount = _attemptCount; return newPtr; - } } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index db3d084e1a..6d3173ba01 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -38,7 +38,6 @@ #include "qmeta/types.h" #include "util/QdispPool.h" #include "wbase/SendChannel.h" -#include "util/InstanceCount.h" namespace lsst::qserv { @@ -143,9 +142,15 @@ class UberJobData : public std::enable_shared_from_this { bool getCancelled() const { return _cancelled; } - /// &&& doc + /// Cancel all Tasks in this UberJob. void cancelAllTasks(); + /// Returns the LIMIT of rows for the query enforceable at the worker, where values <= 0 indicate + /// that there is no limit to the number of rows sent back by the worker. + /// Workers can only safely limit rows for queries that have the LIMIT clause without other related + /// clauses like ORDER BY. + int getRowLimit() { return _rowLimit; } + private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, int czarPort, uint64_t queryId, int rowLimit, uint64_t maxTableSizeBytes, @@ -161,10 +166,6 @@ class UberJobData : public std::enable_shared_from_this { std::string const& url_, std::string const& requestContext_, std::string const& requestStr_); - /// &&& doc - void _queueUJResponse(http::Method method_, std::vector const& headers_, std::string const& url_, std::string const& requestContext_, std::string const& requestStr_); - - UberJobId const _uberJobId; std::string const _czarName; qmeta::CzarId const _czarId; diff --git a/src/wcontrol/Foreman.cc b/src/wcontrol/Foreman.cc index 75f557e7f6..f990667359 100644 --- a/src/wcontrol/Foreman.cc +++ b/src/wcontrol/Foreman.cc @@ -177,8 +177,6 @@ Foreman::~Foreman() { _httpServer->stop(); } -//&&& wpublish::QueryStatistics::Ptr Foreman::addQueryId(QueryId qId) { return _queries->addQueryId(qId); } - void Foreman::processTasks(vector const& tasks) { std::vector cmds; _queries->addTasks(tasks, cmds); diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index e0e49edc2c..a2e381ffae 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -238,7 +238,6 @@ bool QueryRunner::_dispatchChannel() { LOGS(_log, LOG_LVL_TRACE, "QR " << taskSched->histTimeOfRunningTasks->getString("run")); } else { LOGS(_log, LOG_LVL_ERROR, "QR runtaskSched == nullptr"); - LOGS(_log, LOG_LVL_ERROR, "&&&DASH QR runtaskSched == nullptr"); } double runTimeSeconds = primeT.getElapsed(); double subchunkRunTimeSeconds = subChunkT.getElapsed(); diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 184b6c6f87..2dd5bfc60b 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -54,7 +54,6 @@ namespace util = lsst::qserv::util; using lsst::qserv::mysql::MySqlConfig; using lsst::qserv::mysql::MySqlConnection; - using lsst::qserv::wbase::FileChannelShared; using lsst::qserv::wbase::SendChannel; using lsst::qserv::wbase::Task; @@ -139,7 +138,6 @@ struct Fixture { string const czarHostName = "cz5host"; int const czarPort = 3437; string const targWorkerId = "a_worker"; - // &&& make mock foreman instead of nullptr? std::shared_ptr foreman; int const queryId = 23; int const jobId = 1; @@ -151,6 +149,7 @@ struct Fixture { bool const lockInMemory = false; string const resultName = "resName"; string const authKey = "noAuthKey"; + int const rowLimit = 0; }; shared_ptr newTaskJson(MsgInfo const& mInfo) { @@ -172,15 +171,6 @@ struct Fixture { auto& jsJobMsg = *jsJobMsgPtr; auto& chunkScanTables = jsJobMsg["chunkScanTables"]; - /* &&& - for (auto const& sTbl : chunkQuerySpec.scanInfo.infoTables) { - nlohmann::json cst = {{"db", sTbl.db}, - {"table", sTbl.table}, - {"lockInMemory", sTbl.lockInMemory}, - {"tblScanRating", sTbl.scanRating}}; - chunkScanTables.push_back(move(cst)); - } - */ nlohmann::json cst = {{"db", mInfo.db}, {"table", mInfo.table}, {"lockInMemory", mInfo.lockInMemory}, @@ -188,32 +178,6 @@ struct Fixture { chunkScanTables.push_back(move(cst)); auto& jsFragments = jsJobMsg["queryFragments"]; - /* &&& - if (chunkQuerySpec.nextFragment.get()) { - ChunkQuerySpec const* sPtr = &chunkQuerySpec; - while (sPtr) { - LOGS(_log, LOG_LVL_TRACE, "nextFragment"); - for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " q=" << (sPtr->queries).at(t)); - } - for (auto const& sbi : sPtr->subChunkIds) { - LOGS(_log, LOG_LVL_DEBUG, __func__ << " sbi=" << sbi); - } - // Linked fragments will not have valid subChunkTables vectors, - // So, we reuse the root fragment's vector. - _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, sPtr->subChunkIds, - sPtr->queries); - sPtr = sPtr->nextFragment.get(); - } - } else { - LOGS(_log, LOG_LVL_TRACE, "no nextFragment"); - for (unsigned int t = 0; t < (chunkQuerySpec.queries).size(); t++) { - LOGS(_log, LOG_LVL_TRACE, (chunkQuerySpec.queries).at(t)); - } - _addFragmentJson(jsFragments, resultTable, chunkQuerySpec.subChunkTables, - chunkQuerySpec.subChunkIds, chunkQuerySpec.queries); - } - */ nlohmann::json jsFrag = {{"resultTable", mInfo.resultName}, {"queries", nlohmann::json::array()}, {"subchunkTables", nlohmann::json::array()}, @@ -284,17 +248,18 @@ BOOST_AUTO_TEST_CASE(Simple) { auto sChannel = FileChannelShared::create(sendC, mInfo.czarId); FakeBackend::Ptr backend = make_shared(); shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); + SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto ujData = lsst::qserv::wbase::UberJobData::create(mInfo.uberJobId, mInfo.czarName, mInfo.czarId, - mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); - lsst::qserv::proto::ScanInfo scanInfo; - scanInfo.scanRating = mInfo.scanRating; - scanInfo.infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); + scanInfo->scanRating = mInfo.scanRating; + scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); vector taskVect = - Task::createTasksForChunk(ujData, *msgJson, sChannel, scanInfo, mInfo.scanInteractive, - mInfo.maxTableSize, crm, newMySqlConfig(), sqlConnMgr, queries); + Task::createTasksForUnitTest(ujData, *msgJson, sChannel, scanInfo, mInfo.scanInteractive, + mInfo.maxTableSize, crm); + Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); @@ -334,17 +299,18 @@ BOOST_AUTO_TEST_CASE(Output) { auto sc = FileChannelShared::create(sendC, mInfo.czarId); FakeBackend::Ptr backend = make_shared(); shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 15); + SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto ujData = lsst::qserv::wbase::UberJobData::create(mInfo.uberJobId, mInfo.czarName, mInfo.czarId, - mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); - lsst::qserv::proto::ScanInfo scanInfo; - scanInfo.scanRating = mInfo.scanRating; - scanInfo.infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); + scanInfo->scanRating = mInfo.scanRating; + scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); vector taskVect = - Task::createTasksForChunk(ujData, *msgJson, sc, scanInfo, mInfo.scanInteractive, - mInfo.maxTableSize, crm, newMySqlConfig(), sqlConnMgr, queries); + Task::createTasksForUnitTest(ujData, *msgJson, sc, scanInfo, mInfo.scanInteractive, + mInfo.maxTableSize, crm); + Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); BOOST_CHECK(a->runQuery()); diff --git a/src/wdb/testQuerySql.cc b/src/wdb/testQuerySql.cc deleted file mode 100644 index f28d733d65..0000000000 --- a/src/wdb/testQuerySql.cc +++ /dev/null @@ -1,101 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013-2015 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/** - * @brief Simple testing for class QuerySql - * - * @author Daniel L. Wang, SLAC - */ - -// Third-party headers - -// Qserv headers -#include "proto/worker.pb.h" -#include "wdb/QuerySql.h" -#include "wdb/QuerySql_Batch.h" - -// Boost unit test header -#define BOOST_TEST_MODULE QuerySql_1 -#include - -namespace test = boost::test_tools; - -//&&& delete file - -using lsst::qserv::proto::TaskMsg_Fragment; -using lsst::qserv::proto::TaskMsg_Subchunk; -using lsst::qserv::wdb::QuerySql; - -struct Fixture { - Fixture() { - defaultDb = "Winter"; - defaultResult = "myResult"; - } - ~Fixture() {} - - TaskMsg_Fragment makeFragment() { - TaskMsg_Fragment f; - // "Real" subchunk query text should include - // pre-substituted subchunk query text. - f.add_query("SELECT o1.*, o2.* FROM Object_1001 o1, Object_1001 o2;"); - f.set_resulttable("fragResult"); - TaskMsg_Subchunk sc; - sc.set_database("obsolete"); - lsst::qserv::proto::TaskMsg_Subchunk_DbTbl* dbTbl = sc.add_dbtbl(); - dbTbl->set_db(defaultDb); - dbTbl->set_tbl("Object"); - sc.add_id(1111); - sc.add_id(1222); - f.mutable_subchunks()->CopyFrom(sc); - return f; - } - - void printQsql(QuerySql const& q) { std::cout << "qsql=" << q << std::endl; } - std::string defaultDb; - std::string defaultResult; -}; - -BOOST_FIXTURE_TEST_SUITE(QuerySqlSuite, Fixture) - -BOOST_AUTO_TEST_CASE(Basic) { - std::shared_ptr qSql; - TaskMsg_Fragment frag = makeFragment(); - qSql = std::make_shared(defaultDb, 1001, frag, true, defaultResult); - BOOST_CHECK(qSql.get()); - printQsql(*qSql); -} - -BOOST_AUTO_TEST_CASE(QueryBatch) { - std::shared_ptr qSql; - TaskMsg_Fragment frag = makeFragment(); - qSql = std::make_shared(defaultDb, 1001, frag, true, defaultResult); - BOOST_CHECK(qSql.get()); - - QuerySql::Batch build("QueryBuildSub", qSql->buildList); - QuerySql::Batch& batch = build; - while (!batch.isDone()) { - std::string piece = batch.current(); - batch.next(); - } -} - -BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wpublish/QueriesAndChunks.cc b/src/wpublish/QueriesAndChunks.cc index 1410e819d2..0b325fffc3 100644 --- a/src/wpublish/QueriesAndChunks.cc +++ b/src/wpublish/QueriesAndChunks.cc @@ -143,6 +143,7 @@ void QueriesAndChunks::addTask(wbase::Task::Ptr const& task) { auto stats = addQueryId(qid, czId); stats->addTask(task); } + void QueriesAndChunks::addTasks(vector const& tasks, std::vector& cmds) { unique_lock guardStats(_queryStatsMapMtx); diff --git a/src/wpublish/QueryStatistics.cc b/src/wpublish/QueryStatistics.cc index bc06eea581..2ca96d7f37 100644 --- a/src/wpublish/QueryStatistics.cc +++ b/src/wpublish/QueryStatistics.cc @@ -189,13 +189,6 @@ QueryStatistics::SchedTasksInfoMap QueryStatistics::getSchedulerTasksInfoMap() { return _taskSchedInfoMap; } -/* &&& -void QueryStatistics::touch(TIMEPOINT const now) { - lock_guard lock(_qStatsMtx); - _touched = now; -} -*/ - void QueryStatistics::addTask(TIMEPOINT const now) { lock_guard lock(_qStatsMtx); _touched = now; diff --git a/src/wpublish/QueryStatistics.h b/src/wpublish/QueryStatistics.h index 4a366ad5a6..5fd24ff003 100644 --- a/src/wpublish/QueryStatistics.h +++ b/src/wpublish/QueryStatistics.h @@ -41,7 +41,6 @@ #include "global/intTypes.h" #include "wbase/Task.h" #include "wsched/SchedulerBase.h" -#include "util/InstanceCount.h" //&&& namespace lsst::qserv::wbase { class UserQueryInfo; @@ -96,7 +95,6 @@ class QueryStatistics { void addTaskTransmit(double timeSeconds, int64_t bytesTransmitted, int64_t rowsTransmitted, double bufferFillSecs); - //&&&void touch(TIMEPOINT const now); void addTask(TIMEPOINT const now); void addTaskRunning(TIMEPOINT const now); bool addTaskCompleted(TIMEPOINT const now, double const taskDuration); diff --git a/src/wsched/BlendScheduler.cc b/src/wsched/BlendScheduler.cc index 0d1ab2c0a9..525b6afa9f 100644 --- a/src/wsched/BlendScheduler.cc +++ b/src/wsched/BlendScheduler.cc @@ -270,6 +270,7 @@ void BlendScheduler::commandStart(util::Command::Ptr const& cmd) { LOGS(_log, LOG_LVL_ERROR, "BlendScheduler::commandStart scheduler not found"); } _infoChanged = true; + LOGS(_log, LOG_LVL_DEBUG, "BlendScheduler::commandStart &&& end"); } void BlendScheduler::commandFinish(util::Command::Ptr const& cmd) { diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index fcce3157cb..c42321384a 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -36,7 +36,6 @@ #include "http/MetaModule.h" #include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" -#include "http/WorkerQueryStatusData.h" #include "mysql/MySqlUtils.h" #include "protojson/UberJobMsg.h" #include "protojson/WorkerQueryStatusData.h" @@ -250,6 +249,7 @@ json HttpWorkerCzarModule::_handleQueryStatus(std::string const& func) { wCzarInfo->czarMsgReceived(CLOCK::now()); auto const czInfo = wqsData->getCzInfo(); + LOGS(_log, LOG_LVL_TRACE, " HttpWorkerCzarModule::_handleQueryStatus req=" << jsReq.dump()); CzarIdType czId = czInfo->czId; wcontrol::WCzarInfoMap::Ptr wCzarMap = foreman()->getWCzarInfoMap(); wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czId); From f83d0c19d47255760dd09cae5898c7553608cb86 Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 18 Dec 2024 09:37:50 -0800 Subject: [PATCH 08/15] Changed Czar to catch 5GB limit. --- src/ccontrol/MergingHandler.cc | 163 +-------------------- src/ccontrol/MergingHandler.h | 3 - src/ccontrol/UserQuerySelect.cc | 208 --------------------------- src/czar/CzarChunkMap.cc | 12 +- src/czar/HttpCzarWorkerModule.cc | 122 ---------------- src/protojson/testUberJobMsg.cc | 43 ------ src/qana/QueryMapping.h | 2 - src/qdisp/Executive.cc | 24 +--- src/qdisp/JobQuery.h | 1 - src/qdisp/UberJob.cc | 235 +------------------------------ src/qproc/ChunkQuerySpec.h | 3 +- src/qproc/QuerySession.cc | 1 - src/rproc/InfileMerger.cc | 111 --------------- src/wbase/Task.cc | 9 +- src/wbase/Task.h | 15 +- src/wbase/UberJobData.cc | 20 +-- src/wbase/UberJobData.h | 36 +---- src/wconfig/WorkerConfig.h | 51 +++++++ src/wdb/testQueryRunner.cc | 88 ++---------- src/wpublish/QueriesAndChunks.cc | 1 - src/wsched/BlendScheduler.cc | 1 - src/wsched/ChunkTasksQueue.cc | 2 +- 22 files changed, 93 insertions(+), 1058 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 20ee860cac..82f7e9fa26 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -211,9 +211,11 @@ std::tuple readHttpFileAndMergeHttp( } // Parse and evaluate the message. + //&&&mergeHappened = messageIsReady(msgBuf.get(), msgSizeBytes, last); mergeHappened = true; bool messageReadyResult = messageIsReady(msgBuf.get(), msgSizeBytes, last); totalBytesRead += msgSizeBytes; + //&&&if (!mergeHappened) { if (!messageReadyResult) { success = false; throw runtime_error("message processing failed at offset " + @@ -259,166 +261,6 @@ std::tuple readHttpFileAndMergeHttp( return {success, mergeHappened}; } -std::tuple readHttpFileAndMergeHttp( - lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, - function const& messageIsReady, - shared_ptr const& httpConnPool) { - string const context = "MergingHandler::" + string(__func__) + " " + " qid=" + uberJob->getIdStr() + " "; - - LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); - - // Track the file while the control flow is staying within the function. - ResultFileTracker const resultFileTracker; - - // The data transmit rate tracker is set up before reading each data message. - unique_ptr> transmitRateTracker; - - // A location of the next byte to be read from the input file. The variable - // is used for error reporting. - uint64_t offset = 0; - - // Temporary buffer for messages read from the file. The buffer gets automatically - // resized to fit the largest message. - unique_ptr msgBuf; - size_t msgBufSize = 0; - size_t msgBufNext = 0; // An index of the next character in the buffer. - - // Fixed-size buffer to store the message size. - string msgSizeBuf(sizeof(uint32_t), '\0'); - size_t msgSizeBufNext = 0; // An index of the next character in the buffer. - - // The size of the next/current message. The variable is set after succesfully parsing - // the message length header and is reset back to 0 after parsing the message body. - // The value is stays 0 while reading the frame header. - uint32_t msgSizeBytes = 0; - bool success = true; - bool mergeSuccess = true; - int headerCount = 0; - uint64_t totalBytesRead = 0; - try { - string const noClientData; - vector const noClientHeaders; - http::ClientConfig clientConfig; - clientConfig.httpVersion = CURL_HTTP_VERSION_1_1; // same as in qhttp - clientConfig.bufferSize = CURL_MAX_READ_SIZE; // 10 MB in the current version of libcurl - clientConfig.tcpKeepAlive = true; - clientConfig.tcpKeepIdle = 5; // the default is 60 sec - clientConfig.tcpKeepIntvl = 5; // the default is 60 sec - http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, - httpConnPool); - reader.read([&](char const* inBuf, size_t inBufSize) { - // A value of the flag is set by the message processor when it's time to finish - // or abort reading the file. - bool last = false; - char const* next = inBuf; - char const* const end = inBuf + inBufSize; - while ((next < end) && !last) { - LOGS(_log, LOG_LVL_WARN, - context << "TODO:UJ next=" << (uint64_t)next << " end=" << (uint64_t)end - << " last=" << last); - if (msgSizeBytes == 0) { - // Continue or finish reading the frame header. - size_t const bytes2read = - std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); - std::memcpy(msgSizeBuf.data() + msgSizeBufNext, next, bytes2read); - next += bytes2read; - offset += bytes2read; - msgSizeBufNext += bytes2read; - if (msgSizeBufNext == sizeof(uint32_t)) { - ++headerCount; - // Done reading the frame header. - msgSizeBufNext = 0; - // Parse and evaluate the message length. - msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); - if (msgSizeBytes == 0) { - throw runtime_error("message size is 0 at offset " + - to_string(offset - sizeof(uint32_t)) + ", file: " + httpUrl); - } - if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { - throw runtime_error("message size " + to_string(msgSizeBytes) + " at offset " + - to_string(offset - sizeof(uint32_t)) + - " exceeds the hard limit of " + - to_string(ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) + - ", file: " + httpUrl); - } - // Extend the message buffer (if needed). Note that buffer never gets - // truncated to avoid excessive memory deallocations/allocations. - if (msgBufSize < msgSizeBytes) { - msgBufSize = msgSizeBytes; - msgBuf.reset(new char[msgBufSize]); - } - // Starts the tracker to measure the performance of the network I/O. - transmitRateTracker = - make_unique>(reportFileRecvRate); - } - } else { - // Continue or finish reading the message body. - size_t const bytes2read = - std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); - std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); - next += bytes2read; - offset += bytes2read; - msgBufNext += bytes2read; - if (msgBufNext == msgSizeBytes) { - // Done reading message body. - msgBufNext = 0; - - // Destroying the tracker will result in stopping the tracker's timer and - // reporting the file read rate before proceeding to the merge. - if (transmitRateTracker != nullptr) { - transmitRateTracker->addToValue(msgSizeBytes); - transmitRateTracker->setSuccess(); - transmitRateTracker.reset(); - } - - // Parse and evaluate the message. - mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); - totalBytesRead += msgSizeBytes; - if (!mergeSuccess) { - success = false; - throw runtime_error("message processing failed at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - // Reset the variable to prepare for reading the next header & message (if any). - msgSizeBytes = 0; - } else { - LOGS(_log, LOG_LVL_WARN, - context << " headerCount=" << headerCount - << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); - } - } - } - }); - LOGS(_log, LOG_LVL_DEBUG, - context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes - << " totalBytesRead=" << totalBytesRead); - if (msgSizeBufNext != 0) { - throw runtime_error("short read of the message header at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - if (msgBufNext != 0) { - throw runtime_error("short read of the message body at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); - success = false; - } - - // Remove the file from the worker if it still exists. Report and ignore errors. - // The files will be garbage-collected by workers. - try { - http::Client remover(http::Method::DELETE, httpUrl); - remover.read([](char const* inBuf, size_t inBufSize) {}); - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, context << "failed to remove " << httpUrl << ", ex: " << ex.what()); - } - // If the merge failed, that indicates something went wrong in the local database table, - // is likely this user query is doomed and should be cancelled. - LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeSuccess); - return {success, mergeSuccess}; -} - } // namespace namespace lsst::qserv::ccontrol { @@ -426,7 +268,6 @@ namespace lsst::qserv::ccontrol { MergingHandler::MergingHandler(std::shared_ptr merger, std::string const& tableName) : _infileMerger{merger}, _tableName{tableName} {} - MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_TRACE, __func__ << " " << _tableName); } diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 6876e9cd68..7de0e7be9f 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -103,9 +103,6 @@ class MergingHandler : public qdisp::ResponseHandler { /// Call InfileMerger to do the work of merging this data to the result. bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); - /// Call InfileMerger to do the work of merging this data to the result. - bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); - /// Set error code and string. void _setError(int code, std::string const& msg); diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 48ffd39e5b..44180fc4eb 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -106,7 +106,6 @@ #include "sql/Schema.h" #include "util/Bug.h" #include "util/IterableFormatter.h" -#include "util/Histogram.h" //&&& #include "util/QdispPool.h" #include "util/ThreadPriority.h" #include "qdisp/UberJob.h" @@ -235,7 +234,6 @@ void UserQuerySelect::submit() { LOGS(_log, LOG_LVL_ERROR, "UserQuerySelect::submit() executive is null at start"); return; } - _qSession->finalize(); // Using the QuerySession, generate query specs (text, db, chunkId) and then @@ -341,14 +339,6 @@ void UserQuerySelect::buildAndSendUberJobs() { return; } - if (exec->getCancelled() || exec->getSuperfluous()) { - LOGS(_log, LOG_LVL_INFO, funcN << " executive cancelled."); - } - - if (exec->getSuperfluous()) { - LOGS(_log, LOG_LVL_INFO, funcN << " executive superfluous, result already found."); - } - // Only one thread should be generating UberJobs for this user query at any given time. lock_guard fcLock(_buildUberJobMtx); LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << exec->getTotalJobs()); @@ -468,21 +458,6 @@ void UserQuerySelect::buildAndSendUberJobs() { } wInfUJ->wInf = iter->second; } - // Add this job to the appropriate UberJob, making the UberJob if needed. - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c5"); - string workerId = targetWorker->getWorkerId(); - auto& ujVect = workerJobMap[workerId]; - if (ujVect.empty() || ujVect.back()->getJobCount() > maxChunksPerUber) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c5a"); - string uberResultName = ttn.make(uberJobId); - auto respHandler = make_shared(_infileMerger, uberResultName); - auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), uberJobId++, - _qMetaCzarId, targetWorker); - ujVect.push_back(uJob); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c6"); - ujVect.back()->addJob(jqPtr); - } if (wInfUJ->uberJobPtr == nullptr) { auto ujId = _uberJobIdSeq++; // keep ujId consistent @@ -527,189 +502,6 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_DEBUG, funcN << " " << exec->dumpUberJobCounts()); } -util::HistogramRolling histoBuildAndS("&&&uj histoBuildAndS", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); - -void UserQuerySelect::buildAndSendUberJobs() { - // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? - string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); - LOGS(_log, LOG_LVL_DEBUG, funcN << " start " << _uberJobMaxChunks); - - // Ensure `_monitor()` doesn't do anything until everything is ready. - if (!_executive->isReadyToExecute()) { - LOGS(_log, LOG_LVL_INFO, funcN << " executive isn't ready to generate UberJobs."); - return; - } - - // Only one thread should be generating UberJobs for this user query at any given time. - lock_guard fcLock(_buildUberJobMtx); - LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << _executive->getTotalJobs()); - - vector uberJobs; - - qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); - if (unassignedChunksInQuery.empty()) { - LOGS(_log, LOG_LVL_DEBUG, funcN << " no unassigned Jobs"); - return; - } - - // Get czar info and the worker contactMap. - auto czarPtr = czar::Czar::getCzar(); - auto czFamilyMap = czarPtr->getCzarFamilyMap(); - auto czChunkMap = czFamilyMap->getChunkMap(_queryDbName); - auto czRegistry = czarPtr->getCzarRegistry(); - auto const wContactMap = czRegistry->waitForWorkerContactMap(); - - if (czChunkMap == nullptr) { - LOGS(_log, LOG_LVL_ERROR, funcN << " no map found for queryDbName=" << _queryDbName); - // Make an empty chunk map so all jobs are flagged as needing to be reassigned. - // There's a chance that a family will be replicated by the registry. - czChunkMap = czar::CzarChunkMap::create(); - } - - auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); - // Make a map of all jobs in the executive. - // TODO:UJ Maybe a check should be made that all databases are in the same family? - - // keep cycling through workers until no more chunks to place. - // - create a map of UberJobs key=, val=> - // - for chunkId in `unassignedChunksInQuery` - // - use `chunkMapPtr` to find the shared scan workerId for chunkId - // - if not existing in the map, make a new uberjob - // - if existing uberjob at max jobs, create a new uberjob - // - once all chunks in the query have been put in uberjobs, find contact info - // for each worker - // - add worker to each uberjob. - // - For failures - If a worker cannot be contacted, that's an uberjob failure. - // - uberjob failures (due to communications problems) will result in the uberjob - // being broken up into multiple UberJobs going to different workers. - // - If an UberJob fails, the UberJob is killed and all the Jobs it contained - // are flagged as needing re-assignment and this function will be called - // again to put those Jobs in new UberJobs. Correctly re-assigning the - // Jobs requires accurate information from the registry about which workers - // are alive or dead. - struct WInfoAndUJPtr { - using Ptr = shared_ptr; - qdisp::UberJob::Ptr uberJobPtr; - protojson::WorkerContactInfo::Ptr wInf; - }; - map workerJobMap; - vector missingChunks; - - auto startassign = CLOCK::now(); //&&& - // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in - // numerical order. The workers run shared scans in numerical order of chunkId numbers. - // Numerical order keeps the number of partially complete UberJobs running on a worker to a minimum, - // and should minimize the time for the first UberJob on the worker to complete. - for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { - - bool const increaseAttemptCount = true; - jqPtr->getDescription()->incrAttemptCount(_executive, increaseAttemptCount); - - // If too many workers are down, there will be a chunk that cannot be found. - // Just continuing should leave jobs `unassigned` with their attempt count - // increased. Either the chunk will be found and jobs assigned, or the jobs' - // attempt count will reach max and the query will be cancelled - auto lambdaMissingChunk = [&](string const& msg) { - missingChunks.push_back(chunkId); - //&&&bool const increaseAttemptCount = true; - //&&&jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); - LOGS(_log, LOG_LVL_ERROR, msg); - }; - - auto iter = chunkMapPtr->find(chunkId); - if (iter == chunkMapPtr->end()) { - lambdaMissingChunk(funcN + " No chunkData for=" + to_string(chunkId)); - continue; - } - czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; - auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - // TODO:UJ maybe if (targetWorker == nullptr || ... || this worker already tried for this chunk) { - if (targetWorker == nullptr || targetWorker->isDead()) { - LOGS(_log, LOG_LVL_WARN, - funcN << " No primary scan worker for chunk=" + chunkData->dump() - << ((targetWorker == nullptr) ? " targ was null" : " targ was dead")); - // Try to assign a different worker to this job - auto workerHasThisChunkMap = chunkData->getWorkerHasThisMapCopy(); - bool found = false; - for (auto wIter = workerHasThisChunkMap.begin(); wIter != workerHasThisChunkMap.end() && !found; - ++wIter) { - auto maybeTarg = wIter->second.lock(); - if (maybeTarg != nullptr && !maybeTarg->isDead()) { - targetWorker = maybeTarg; - found = true; - LOGS(_log, LOG_LVL_WARN, - funcN << " Alternate worker=" << targetWorker->getWorkerId() - << " found for chunk=" << chunkData->dump()); - } - } - if (!found) { - lambdaMissingChunk(funcN + - " No primary or alternate worker found for chunk=" + chunkData->dump()); - continue; - } - } - // Add this job to the appropriate UberJob, making the UberJob if needed. - string workerId = targetWorker->getWorkerId(); - WInfoAndUJPtr::Ptr& wInfUJ = workerJobMap[workerId]; - if (wInfUJ == nullptr) { - wInfUJ = make_shared(); - auto iter = wContactMap->find(workerId); - if (iter == wContactMap->end()) { - // TODO:UJ Not appropriate to throw for this. Need to re-direct all jobs to different workers. - // Also, this really shouldn't happen, but crashing the czar is probably a bad idea, - // so maybe return internal error to the user? - throw util::Bug(ERR_LOC, funcN + " TODO:UJ no contact information for " + workerId); - } - wInfUJ->wInf = iter->second; - } - - if (wInfUJ->uberJobPtr == nullptr) { - auto ujId = _uberJobIdSeq++; // keep ujId consistent - string uberResultName = _ttn->make(ujId); - auto respHandler = make_shared(_infileMerger, uberResultName); - auto uJob = qdisp::UberJob::create(_executive, respHandler, _executive->getId(), ujId, - _qMetaCzarId, targetWorker); - uJob->setWorkerContactInfo(wInfUJ->wInf); - wInfUJ->uberJobPtr = uJob; - }; - - wInfUJ->uberJobPtr->addJob(jqPtr); - - if (wInfUJ->uberJobPtr->getJobCount() >= _uberJobMaxChunks) { - // Queue the UberJob to be sent to a worker - _executive->addAndQueueUberJob(wInfUJ->uberJobPtr); - - // Clear the pinter so a new UberJob is created later if needed. - wInfUJ->uberJobPtr = nullptr; - } - } - auto endassign = CLOCK::now(); //&&& - std::chrono::duration secsassign = endassign - startassign; // &&& - histoBuildAndS.addEntry(endassign, secsassign.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoBuildAndS.getString("")); - - if (!missingChunks.empty()) { - string errStr = funcN + " a worker could not be found for these chunks "; - for (auto const& chk : missingChunks) { - errStr += to_string(chk) + ","; - } - errStr += " they will be retried later."; - LOGS(_log, LOG_LVL_ERROR, errStr); - } - - // Queue unqued UberJobs, these have less than the max number of jobs. - for (auto const& [wIdKey, winfUjPtr] : workerJobMap) { - if (winfUjPtr != nullptr) { - auto& ujPtr = winfUjPtr->uberJobPtr; - if (ujPtr != nullptr) { - _executive->addAndQueueUberJob(ujPtr); - } - } - } - - LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); -} - /// Block until a submit()'ed query completes. /// @return the QueryState indicating success or failure QueryState UserQuerySelect::join() { diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 33afa5dd4f..c1ebad63eb 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -36,8 +36,6 @@ #include "czar/CzarRegistry.h" #include "qmeta/Exceptions.h" #include "util/Bug.h" -#include "util/InstanceCount.h" //&&& -#include "util/Histogram.h" //&&& #include "util/TimeUtils.h" using namespace std; @@ -334,7 +332,7 @@ bool CzarFamilyMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarFamilyMap::_read() start"); // If replacing the map, this may take a bit of time, but it's probably // better to wait for new maps if something changed. - std::lock_guard gLock(_familyMapMtx); // &&& check waiting is really needed + std::lock_guard gLock(_familyMapMtx); qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(_lastUpdateTime); if (_lastUpdateTime == qChunkMap.updateTime) { LOGS(_log, LOG_LVL_DEBUG, @@ -359,13 +357,9 @@ bool CzarFamilyMap::_read() { return true; } -util::HistogramRolling histoMakeNewMaps("&&&uj histoMakeNewMaps", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); - std::shared_ptr CzarFamilyMap::makeNewMaps( qmeta::QMetaChunkMap const& qChunkMap, bool usingChunkSize) { // Create new maps. - util::InstanceCount ic("CzarFamilyMap::makeNewMaps&&&"); - auto startMakeMaps = CLOCK::now(); //&&& std::shared_ptr newFamilyMap = make_shared(); // Workers -> Databases map @@ -422,10 +416,6 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( } } - auto endMakeMaps = CLOCK::now(); //&&& - std::chrono::duration secsMakeMaps = endMakeMaps - startMakeMaps; // &&& - histoMakeNewMaps.addEntry(endMakeMaps, secsMakeMaps.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoMakeNewMaps.getString("")); return newFamilyMap; } diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 9b1f5b783e..9d134aead5 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -225,126 +225,4 @@ json HttpCzarWorkerModule::_handleWorkerCzarComIssue(string const& func) { return jsRet; } -json HttpCzarWorkerModule::_handleJobError(string const& func) { - // Metadata-only responses for the file-based protocol should not have any data - - // Parse and verify the json message and then kill the UberJob. - json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; - try { - // See qdisp::UberJob::runUberJob() for json message construction. &&& - string const targetWorkerId = body().required("workerid"); - string const czarName = body().required("czar"); - qmeta::CzarId const czarId = body().required("czarid"); - QueryId const queryId = body().required("queryid"); - UberJobId const uberJobId = body().required("uberjobid"); - int const errorCode = body().required("errorCode"); - string const errorMsg = body().required("errorMsg"); - - // Find UberJob - qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); - if (exec == nullptr) { - throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No executive for qid=") + - to_string(queryId) + " czar=" + to_string(czarId)); - } - qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); - if (uj == nullptr) { - throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No UberJob for qid=") + - to_string(queryId) + " ujId=" + to_string(uberJobId) + - " czar=" + to_string(czarId)); - } - - auto importRes = uj->workerError(errorCode, errorMsg); - jsRet = importRes; - - } catch (std::invalid_argument const& iaEx) { - LOGS(_log, LOG_LVL_ERROR, - "HttpCzarWorkerModule::_handleJobError received " << iaEx.what() << " js=" << body().objJson); - jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; - } - LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobError end"); - return jsRet; -} - -json HttpCzarWorkerModule::_handleJobReady(string const& func) { - LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady start"); - // Metadata-only responses for the file-based protocol should not have any data - - // Parse and verify the json message and then have the uberjob import the file. - json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; - try { - // &&& TODO:UJ file response - move construction and parsing - // &&& TODO:UJ to a class so it can be added to WorkerCzarComIssue - // See qdisp::UberJob::runUberJob() for json message construction. &&& - string const targetWorkerId = body().required("workerid"); - string const czarName = body().required("czar"); - qmeta::CzarId const czarId = body().required("czarid"); - QueryId const queryId = body().required("queryid"); - UberJobId const uberJobId = body().required("uberjobid"); - string const fileUrl = body().required("fileUrl"); - uint64_t const rowCount = body().required("rowCount"); - uint64_t const fileSize = body().required("fileSize"); - - // Find UberJob - qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); - if (exec == nullptr) { - throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + - to_string(queryId) + " czar=" + to_string(czarId)); - } - qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); - if (uj == nullptr) { - throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No UberJob for qid=") + - to_string(queryId) + " ujId=" + to_string(uberJobId) + - " czar=" + to_string(czarId)); - } - - auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); - jsRet = importRes; - - } catch (std::invalid_argument const& iaEx) { - LOGS(_log, LOG_LVL_ERROR, - "HttpCzarWorkerModule::_handleJobReady received " << iaEx.what() << " js=" << body().objJson); - jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; - } - LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady end"); - return jsRet; -} - -json HttpCzarWorkerModule::_handleWorkerCzarComIssue(string const& func) { - LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleWorkerCzarComIssue start"); - // Parse and verify the json message and then deal with the problems. - json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; - try { - string const replicationInstanceId = cconfig::CzarConfig::instance()->replicationInstanceId(); - string const replicationAuthKey = cconfig::CzarConfig::instance()->replicationAuthKey(); - auto const& jsReq = body().objJson; - auto wccIssue = protojson::WorkerCzarComIssue::createFromJson(jsReq, replicationInstanceId, - replicationAuthKey); - - auto wId = wccIssue->getWorkerInfo()->wId; - if (wccIssue->getThoughtCzarWasDead()) { - LOGS(_log, LOG_LVL_WARN, - "HttpCzarWorkerModule::_handleWorkerCzarComIssue worker=" - << wId << " thought czar was dead and killed related uberjobs."); - - // Find all incomplete UberJobs with this workerId and re-assign them. - // Use a copy to avoid mutex issues. - auto execMap = czar::Czar::getCzar()->getExecMapCopy(); - for (auto const& [exKey, execWeak] : execMap) { - auto execPtr = execWeak.lock(); - if (execPtr == nullptr) continue; - execPtr->killIncompleteUberJobsOnWorker(wId); - } - } - jsRet = wccIssue->serializeResponseJson(); - LOGS(_log, LOG_LVL_TRACE, "HttpCzarWorkerModule::_handleWorkerCzarComIssue jsRet=" << jsRet.dump()); - - } catch (std::invalid_argument const& iaEx) { - LOGS(_log, LOG_LVL_ERROR, - "HttpCzarWorkerModule::_handleWorkerCzarComIssue received " << iaEx.what() - << " js=" << body().objJson); - jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaEx.what()}}; - } - return jsRet; -} - } // namespace lsst::qserv::czar diff --git a/src/protojson/testUberJobMsg.cc b/src/protojson/testUberJobMsg.cc index baf480feaa..95f8d8627f 100644 --- a/src/protojson/testUberJobMsg.cc +++ b/src/protojson/testUberJobMsg.cc @@ -59,49 +59,6 @@ string testB() { return tb; } -bool parseSerializeReparseCheck(string const& jsStr, string const& note) { - string fName("parseSerialize "); - fName += note + " "; - LOGS(_log, LOG_LVL_INFO, fName << " start " << jsStr); - nlohmann::json js = nlohmann::json::parse(jsStr); - LOGS(_log, LOG_LVL_INFO, fName << " parse 1"); - - UberJobMsg::Ptr ujm = UberJobMsg::createFromJson(js); - BOOST_REQUIRE(ujm != nullptr); - - nlohmann::json jsUjm = ujm->serializeJson(); - LOGS(_log, LOG_LVL_INFO, fName << " serialized jsUjm=" << jsUjm); - - UberJobMsg::Ptr ujmCreated = UberJobMsg::createFromJson(jsUjm); - LOGS(_log, LOG_LVL_INFO, fName << " created"); - nlohmann::json jsUjmCreated = ujmCreated->serializeJson(); - LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); - - bool createdMatchesOriginal = jsUjm == jsUjmCreated; - if (createdMatchesOriginal) { - LOGS(_log, LOG_LVL_INFO, fName << "created matches original"); - } else { - LOGS(_log, LOG_LVL_ERROR, "jsUjm != jsUjmCreated"); - LOGS(_log, LOG_LVL_ERROR, "jsUjm=" << jsUjm); - LOGS(_log, LOG_LVL_ERROR, "jsUjmCreated=" << jsUjmCreated); - } - BOOST_REQUIRE(createdMatchesOriginal); - return createdMatchesOriginal; -} -#endif // &&& - -string testA() { - string ta = - R"({"maxtablesizemb":5432,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; - return ta; -} - -string testB() { - string tb = - R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":39,"worker":"db04"})"; - return tb; -} - bool parseSerializeReparseCheck(string const& jsStr, string const& note) { string fName("parseSerialize "); fName += note + " "; diff --git a/src/qana/QueryMapping.h b/src/qana/QueryMapping.h index 585971f976..2e8dca319b 100644 --- a/src/qana/QueryMapping.h +++ b/src/qana/QueryMapping.h @@ -92,8 +92,6 @@ class QueryMapping { bool hasParameter(Parameter p) const; DbTableSet const& getSubChunkTables() const { return _subChunkTables; } - std::string dump() const { return std::string("&&& NEED CODE"); } - private: ParameterMap _subs; DbTableSet _subChunkTables; diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 9fd46e807a..21b22f17b8 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -261,25 +261,6 @@ void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { } } -void Executive::addAndQueueUberJob(shared_ptr const& uj) { - { - lock_guard lck(_uberJobsMapMtx); - UberJobId ujId = uj->getUjId(); - _uberJobsMap[ujId] = uj; - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); - } - - auto runUberJobFunc = [uj](util::CmdData*) { uj->runUberJob(); }; - - auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(runUberJobFunc)); - _jobStartCmdList.push_back(cmd); - if (_scanInteractive) { - _qdispPool->queCmd(cmd, 0); - } else { - _qdispPool->queCmd(cmd, 1); - } -} - void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { if (_scanInteractive) { _qdispPool->queCmd(cmd, 3); @@ -291,10 +272,9 @@ void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { void Executive::addAndQueueUberJob(shared_ptr const& uj) { { lock_guard lck(_uberJobsMapMtx); - UberJobId ujId = uj->getJobId(); + UberJobId ujId = uj->getUjId(); _uberJobsMap[ujId] = uj; - //&&&uj->setAdded(); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uj->getJobCount()); } auto runUberJobFunc = [uj](util::CmdData*) { uj->runUberJob(); }; diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index c4092e34fd..607c76536d 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -60,7 +60,6 @@ class JobQuery { QueryId getQueryId() const { return _qid; } JobId getJobId() const { return _jobDescription->id(); } std::string const& getIdStr() const { return _idStr; } - JobDescription::Ptr getDescription() { return _jobDescription; } qmeta::JobStatus::Ptr getStatus() { return _jobStatus; } bool cancel(bool superfluous = false); diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 7cc95528dc..299b697ef7 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -43,7 +43,6 @@ #include "qproc/ChunkQuerySpec.h" #include "util/Bug.h" #include "util/common.h" -#include "util/Histogram.h" //&&& #include "util/QdispPool.h" // LSST headers @@ -101,12 +100,8 @@ bool UberJob::addJob(JobQuery::Ptr const& job) { return success; } -util::HistogramRolling histoRunUberJob("&&&uj histoRunUberJob", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); -util::HistogramRolling histoUJSerialize("&&&uj histoUJSerialize", {0.1, 1.0, 10.0, 100.0, 1000.0}, 1h, 10000); - -void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelled +void UberJob::runUberJob() { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << "&&&uj runuj start"); // Build the uberjob payload for each job. nlohmann::json uj; unique_lock jobsLock(_jobsMtx); @@ -168,10 +163,6 @@ void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelle LOGS(_log, LOG_LVL_WARN, requestContext + " ujresponse failed, ex: " + ex.what()); exceptionWhat = ex.what(); } - auto endclient = CLOCK::now(); //&&& - std::chrono::duration secsclient = endclient - startclient; // &&& - histoRunUberJob.addEntry(endclient, secsclient.count()); //&&& - LOGS(_log, LOG_LVL_INFO, "&&&uj histo " << histoRunUberJob.getString("")); if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure, try to send jobs elsewhere"); _unassignJobs(); // locks _jobsMtx @@ -180,7 +171,6 @@ void UberJob::runUberJob() { // &&& TODO:UJ this should probably check cancelle } else { setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx } - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " &&&uj runuj end"); return; } @@ -224,7 +214,7 @@ void UberJob::_unassignJobs() { } for (auto&& job : _jobs) { string jid = job->getIdStr(); - if (!job->unassignFromUberJob(getJobId())) { + if (!job->unassignFromUberJob(getUjId())) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not unassign job=" << jid << " cancelling"); exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "unable to re-assign " + jid, util::ErrorCode::INTERNAL); @@ -545,227 +535,6 @@ void UberJob::killUberJob() { return; } -/// Retrieve and process a result file using the file-based protocol -/// Uses a copy of JobQuery::Ptr instead of _jobQuery as a call to cancel() would reset _jobQuery. -json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { - LOGS(_log, LOG_LVL_DEBUG, - cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); - - if (isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); - return _importResultError(true, "cancelled", "Query cancelled"); - } - - auto exec = _executive.lock(); - if (exec == nullptr || exec->getCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive or cancelled"); - return _importResultError(true, "cancelled", "Query cancelled - no executive"); - } - - if (exec->isRowLimitComplete()) { - int dataIgnored = exec->incrDataIgnoredCount(); - if ((dataIgnored - 1) % 1000 == 0) { - LOGS(_log, LOG_LVL_INFO, - "UberJob ignoring, enough rows already " << "dataIgnored=" << dataIgnored); - } - return _importResultError(false, "rowLimited", "Enough rows already"); - } - - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileSize=" << fileSize); - - bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); - if (!statusSet) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " setStatusFail could not set status to RESPONSE_READY"); - return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); - } - - weak_ptr ujThis = weak_from_this(); - // TODO:UJ lambda may not be the best way to do this, alsocheck synchronization - may need a mutex for - // merging. - string const idStr = _idStr; - auto fileCollectFunc = [ujThis, fileUrl, rowCount, idStr](util::CmdData*) { - auto ujPtr = ujThis.lock(); - if (ujPtr == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, - "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); - return; - } - uint64_t resultRows = 0; - auto [flushSuccess, flushShouldCancel] = - ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); - LOGS(_log, LOG_LVL_DEBUG, ujPtr->cName(__func__) << "::fileCollectFunc"); - if (!flushSuccess) { - // This would probably indicate malformed file+rowCount or - // writing the result table failed. - ujPtr->_importResultError(flushShouldCancel, "mergeError", "merging failed"); - } - - // At this point all data for this job have been read, there's no point in - // having XrdSsi wait for anything. - ujPtr->_importResultFinish(resultRows); - }; - - auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); - exec->queueFileCollect(cmd); - - // If the query meets the limit row complete complete criteria, it will start - // squashing superfluous results so the answer can be returned quickly. - - json jsRet = {{"success", 1}, {"errortype", ""}, {"note", "queued for collection"}}; - return jsRet; -} - -json UberJob::workerError(int errorCode, string const& errorMsg) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " errcode=" << errorCode << " errmsg=" << errorMsg); - - bool const deleteData = true; - bool const keepData = !deleteData; - auto exec = _executive.lock(); - if (exec == nullptr || isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); - return _workerErrorFinish(deleteData, "cancelled"); - } - - if (exec->isRowLimitComplete()) { - int dataIgnored = exec->incrDataIgnoredCount(); - if ((dataIgnored - 1) % 1000 == 0) { - LOGS(_log, LOG_LVL_INFO, - cName(__func__) << " ignoring, enough rows already " - << "dataIgnored=" << dataIgnored); - } - return _workerErrorFinish(keepData, "none", "rowLimitComplete"); - } - - // Currently there are no detectable recoverable errors from workers. The only - // error that a worker could send back that may possibly be recoverable would - // be a missing table error, which is not trivial to detect. A worker local - // database error may also qualify. - // TODO:UJ see if recoverable errors can be detected on the workers, or - // maybe allow a single retry before sending the error back to the user? - bool recoverableError = false; - recoverableError = true; // TODO:UJ delete after testing - if (recoverableError) { - // The czar should have new maps before the the new UberJob(s) for - // these Jobs are created. (see Czar::_monitor) - _unassignJobs(); - } else { - // Get the error message to the user and kill the user query. - int errState = util::ErrorCode::MYSQLEXEC; - getRespHandler()->flushHttpError(errorCode, errorMsg, errState); - exec->addMultiError(errorCode, errorMsg, errState); - exec->squash(); - } - - string errType = to_string(errorCode) + ":" + errorMsg; - return _workerErrorFinish(deleteData, errType, ""); -} - -json UberJob::_importResultError(bool shouldCancel, string const& errorType, string const& note) { - json jsRet = {{"success", 0}, {"errortype", errorType}, {"note", note}}; - // In all cases, the worker should delete the file as this czar will not ask for it. - - auto exec = _executive.lock(); - if (exec != nullptr) { - LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " shouldCancel=" << shouldCancel << " errorType=" << errorType << " " - << note); - if (shouldCancel) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failing jobs"); - callMarkCompleteFunc(false); // all jobs failed, no retry - exec->squash(); - } else { - /// - each JobQuery in _jobs needs to be flagged as needing to be - /// put in an UberJob and it's attempt count increased and checked - /// against the attempt limit. - /// - executive needs to be told to make new UberJobs until all - /// JobQueries are being handled by an UberJob. - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " reassigning jobs"); - _unassignJobs(); - exec->assignJobsToUberJobs(); - } - } else { - LOGS(_log, LOG_LVL_INFO, - cName(__func__) << " already cancelled shouldCancel=" << shouldCancel - << " errorType=" << errorType << " " << note); - } - return jsRet; -} - -void UberJob::_importResultFinish(uint64_t resultRows) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); - - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); - return; - } - - /// If this is called, the file has been collected and the worker should delete it - /// - /// This function should call markComplete for all jobs in the uberjob - /// and return a "success:1" json message to be sent to the worker. - bool const statusSet = - setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); - if (!statusSet) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to set status, squashing " << getIdStr()); - // Something has gone very wrong - exec->squash(); - return; - } - - bool const success = true; - callMarkCompleteFunc(success); // sets status to COMPLETE - exec->addResultRows(resultRows); - exec->checkLimitRowComplete(); -} - -nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, - std::string const& note) { - // If this is called, the file has been collected and the worker should delete it - // - // Should this call markComplete for all jobs in the uberjob??? - // TODO:UJ Only recoverable errors would be: communication failure, or missing table ??? - // Return a "success:1" json message to be sent to the worker. - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); - return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; - } - - json jsRet = {{"success", 1}, {"deletedata", deleteData}, {"errortype", ""}, {"note", ""}}; - return jsRet; -} - -void UberJob::killUberJob() { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " stopping this UberJob and re-assigning jobs."); - - auto exec = _executive.lock(); - if (exec == nullptr || isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); - return; - } - - if (exec->isRowLimitComplete()) { - int dataIgnored = exec->incrDataIgnoredCount(); - if ((dataIgnored - 1) % 1000 == 0) { - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already."); - } - return; - } - - // Put this UberJob on the list of UberJobs that the worker should drop. - auto activeWorkerMap = czar::Czar::getCzar()->getActiveWorkerMap(); - auto activeWorker = activeWorkerMap->getActiveWorker(_wContactInfo->wId); - if (activeWorker != nullptr) { - activeWorker->addDeadUberJob(_queryId, _uberJobId); - } - - _unassignJobs(); - // Let Czar::_monitor reassign jobs - other UberJobs are probably being killed - // so waiting probably gets a better distribution. - return; -} - std::ostream& UberJob::dumpOS(std::ostream& os) const { os << "(jobs sz=" << _jobs.size() << "("; lock_guard lockJobsMtx(_jobsMtx); diff --git a/src/qproc/ChunkQuerySpec.h b/src/qproc/ChunkQuerySpec.h index f9924e9bc8..ef0d8325c4 100644 --- a/src/qproc/ChunkQuerySpec.h +++ b/src/qproc/ChunkQuerySpec.h @@ -65,8 +65,7 @@ class ChunkQuerySpec { bool scanInteractive{false}; DbTableSet subChunkTables; std::vector subChunkIds; - std::vector queries; // &&& remove if possible - std::vector queryTemplates; + std::vector queries; // Consider promoting the concept of container of ChunkQuerySpec // in the hopes of increased code cleanliness. std::shared_ptr nextFragment; ///< ad-hoc linked list (consider removal) diff --git a/src/qproc/QuerySession.cc b/src/qproc/QuerySession.cc index 9bd643265b..969409a4dc 100644 --- a/src/qproc/QuerySession.cc +++ b/src/qproc/QuerySession.cc @@ -391,7 +391,6 @@ std::vector QuerySession::_buildChunkQueries(query::QueryTemplate:: } for (auto&& queryTemplate : queryTemplates) { - LOGS(_log, LOG_LVL_WARN, "&&&uj QuerySession::_buildChunkQueries qt=" << queryTemplate.dump()); std::string str = _context->queryMapping->apply(chunkSpec, queryTemplate); chunkQueries.push_back(std::move(str)); } diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index ed60a7fe85..bb21066f70 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -277,117 +277,6 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response return ret; } -bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { - UberJobId const uJobId = uberJob->getJobId(); - std::string queryIdJobStr = uberJob->getIdStr(); - if (!_queryIdStrSet) { - _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); - } - - // Nothing to do if size is zero. - if (responseData.row_size() == 0) { - return true; - } - - // Do nothing if the query got cancelled for any reason. - if (uberJob->isQueryCancelled()) { - return true; - } - auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { - return true; - } - - std::unique_ptr semaLock; - if (_dbEngine != MYISAM) { - // needed for parallel merging with INNODB and MEMORY - semaLock.reset(new util::SemaLock(*_semaMgrConn)); - } - - TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, - bool success) { - if (!success) return; - if (std::chrono::duration const seconds = end - start; seconds.count() > 0) { - qdisp::CzarStats::get()->addXRootDSSIRecvRate(bytes / seconds.count()); - } - }; - auto tct = make_shared>(cbf); - - bool ret = false; - // Add columns to rows in virtFile. - util::Timer virtFileT; - virtFileT.start(); - // UberJobs only get one attempt - int resultJobId = makeJobIdAttempt(uberJob->getJobId(), 0); - ProtoRowBuffer::Ptr pRowBuffer = std::make_shared( - responseData, resultJobId, _jobIdColName, _jobIdSqlType, _jobIdMysqlType); - std::string const virtFile = _infileMgr.prepareSrc(pRowBuffer); - std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); - virtFileT.stop(); - - // If the job attempt is invalid, exit without adding rows. - // It will wait here if rows need to be deleted. - if (_invalidJobAttemptMgr.incrConcurrentMergeCount(resultJobId)) { - return true; - } - - size_t const resultSize = responseData.transmitsize(); - size_t tResultSize; - { - std::lock_guard resultSzLock(_mtxResultSizeMtx); - _perJobResultSize[uJobId] += resultSize; - tResultSize = _totalResultSize + _perJobResultSize[uJobId]; - } - if (tResultSize > _maxResultTableSizeBytes) { - std::ostringstream os; - os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable - << " is too large at " << tResultSize << " bytes, max allowed size is " << _maxResultTableSizeBytes - << " bytes"; - LOGS(_log, LOG_LVL_ERROR, os.str()); - _error = util::Error(-1, os.str(), -1); - return false; - } - - tct->addToValue(resultSize); - tct->setSuccess(); - tct.reset(); // stop transmit recieve timer before merging happens. - - qdisp::CzarStats::get()->addTotalBytesRecv(resultSize); - qdisp::CzarStats::get()->addTotalRowsRecv(responseData.rowcount()); - - // Stop here (if requested) after collecting stats on the amount of data collected - // from workers. - if (_config.debugNoMerge) { - return true; - } - - auto start = std::chrono::system_clock::now(); - switch (_dbEngine) { - case MYISAM: - ret = _applyMysqlMyIsam(infileStatement, resultSize); - break; - case INNODB: // Fallthrough - case MEMORY: - ret = _applyMysqlInnoDb(infileStatement, resultSize); - break; - default: - throw std::invalid_argument("InfileMerger::_dbEngine is unknown =" + engineToStr(_dbEngine)); - } - auto end = std::chrono::system_clock::now(); - auto mergeDur = std::chrono::duration_cast(end - start); - LOGS(_log, LOG_LVL_DEBUG, - "mergeDur=" << mergeDur.count() << " sema(total=" << _semaMgrConn->getTotalCount() - << " used=" << _semaMgrConn->getUsedCount() << ")"); - if (not ret) { - LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); - } - _invalidJobAttemptMgr.decrConcurrentMergeCount(); - - LOGS(_log, LOG_LVL_DEBUG, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); - - return ret; -} - bool InfileMerger::_applyMysqlMyIsam(std::string const& query, size_t resultSize) { std::unique_lock lock(_mysqlMutex); for (int j = 0; !_mysqlConn.connected(); ++j) { diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 8d231a375a..d4deea6097 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -70,6 +70,7 @@ using namespace nlohmann; namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.Task"); + size_t const MB_SIZE_BYTES = 1024 * 1024; } // namespace @@ -110,12 +111,10 @@ atomic taskSequence{0}; ///< Unique identifier source for Task. /// the util::CommandThreadPool is not called here. Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, size_t templateId, bool hasSubchunks, int subchunkId, string const& db, - vector const& fragSubTables, vector const& fragSubchunkIds, + vector const& fragSubTables, vector const& fragSubchunkIds, shared_ptr const& sc, std::shared_ptr const& queryStats_) - : _logLvlWT(LOG_LVL_WARN), - _logLvlET(LOG_LVL_ERROR), - _sendChannel(sc), + : _sendChannel(sc), _tSeq(++taskSequence), _qId(ujData->getQueryId()), _templateId(templateId), @@ -170,7 +169,6 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " created"); } - Task::~Task() {} std::vector Task::createTasksFromUberJobMsg( @@ -245,7 +243,6 @@ std::vector Task::createTasksFromUberJobMsg( auto task = Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, templateId, noSubchunks, subchunkId, chunkQuerySpecDb, fragSubTables, fragSubchunkIds, sendChannel, queryStats)); - vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { diff --git a/src/wbase/Task.h b/src/wbase/Task.h index ff184d0a93..cc5c7b5814 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -175,24 +175,12 @@ class Task : public util::CommandForThreadPool { mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& sqlConnMgr, std::shared_ptr const& queriesAndChunks); - /// Create Tasks needed to run unit tests. + /// Create Tasks needed to run unit tests. static std::vector createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, std::shared_ptr const& sendChannel, int maxTableSizeMb, std::shared_ptr const& chunkResourceMgr); - //&&& - static std::vector createTasksForUnitTest( - std::shared_ptr const& ujData, nlohmann::json const& jsJobs, - std::shared_ptr const& sendChannel, - protojson::ScanInfo::Ptr const& scanInfo, bool scanInteractive, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr - //&&&mysql::MySqlConfig const& mySqlConfig, std::shared_ptr const& - // sqlConnMgr, - //&&&std::shared_ptr const& queriesAndChunks, - //&&&uint16_t resultsHttpPort = 8080); - ); - std::shared_ptr getSendChannel() const { return _sendChannel; } std::string user; ///< Incoming username // Note that manpage spec of "26 bytes" is insufficient @@ -223,7 +211,6 @@ class Task : public util::CommandForThreadPool { TaskState state() const { return _state; } std::string getQueryString() const; - bool setTaskQueryRunner( TaskQueryRunner::Ptr const& taskQueryRunner); ///< return true if already cancelled. void freeTaskQueryRunner(TaskQueryRunner* tqr); diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 6e6b7f67b9..fc17dc248d 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -26,6 +26,7 @@ // System headers // Third party headers +#include "boost/filesystem.hpp" // LSST headers #include "lsst/log/Log.h" @@ -346,11 +347,12 @@ void UberJobData::cancelAllTasks() { string UJTransmitCmd::cName(const char* funcN) const { stringstream os; - os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " qId=" << _queryId << " ujId=" << _uberJobId; + os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " QID=" << _queryId << "_ujId=" << _uberJobId; return os.str(); } void UJTransmitCmd::action(util::CmdData* data) { + LOGS(_log, LOG_LVL_TRACE, cName(__func__)); // Make certain _selfPtr is reset before leaving this function. // If a retry is needed, duplicate() is called. class ResetSelf { @@ -391,12 +393,12 @@ void UJTransmitCmd::action(util::CmdData* data) { auto wCzInfo = _foreman->getWCzarInfoMap()->getWCzarInfo(_czarId); // This will check if the czar is believed to be alive and try the queue the query to be tried // again at a lower priority. It it thinks the czar is dead, it will throw it away. - // TODO:UJ &&& I have my doubts about this as a reconnected czar may go down in flames - // &&& as it is hit with thousands of these. - // &&& Alternate plan, set a flag in the status message response (WorkerQueryStatusData) - // &&& indicates some messages failed. When the czar sees the flag, it'll request a - // &&& message from the worker that contains all of the failed transmit data and handle - // &&& that. All of these failed transmits should fit in a single message. + // TODO:UJ I have my doubts about this as a reconnected czar may go down in flames + // as it is hit with thousands of these. + // Alternate plan, set a flag in the status message response (WorkerQueryStatusData) + // indicates some messages failed. When the czar sees the flag, it'll request a + // message from the worker that contains all of the failed transmit data and handle + // that. All of these failed transmits should fit in a single message. if (wCzInfo->checkAlive(CLOCK::now())) { auto wPool = _foreman->getWPool(); if (wPool != nullptr) { @@ -419,8 +421,7 @@ void UJTransmitCmd::action(util::CmdData* data) { } void UJTransmitCmd::kill() { - string const funcN("UJTransmitCmd::kill"); - LOGS(_log, LOG_LVL_WARN, funcN); + LOGS(_log, LOG_LVL_WARN, cName(__func__)); auto sPtr = _selfPtr; _selfPtr.reset(); if (sPtr == nullptr) { @@ -429,6 +430,7 @@ void UJTransmitCmd::kill() { } UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { + LOGS(_log, LOG_LVL_INFO, cName(__func__)); auto ujD = _ujData.lock(); if (ujD == nullptr) { return nullptr; diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index 6d3173ba01..5922edeeaf 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -105,7 +105,7 @@ class UberJobData : public std::enable_shared_from_this { uint64_t headerCount); // TODO:UJ remove headerCount /// Let the Czar know there's been a problem. - void responseError(util::MultiError& multiErr, int chunkId, bool cancelled, int logLvl); + bool responseError(util::MultiError& multiErr, int chunkId, bool cancelled); std::string const& getIdStr() const { return _idStr; } std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } @@ -115,42 +115,16 @@ class UberJobData : public std::enable_shared_from_this { /// Cancel all Tasks in this UberJob. void cancelAllTasks(); - /// Returns the LIMIT of rows for the query enforceable at the worker, where values <= 0 indicate - /// that there is no limit to the number of rows sent back by the worker. - /// Workers can only safely limit rows for queries that have the LIMIT clause without other related - /// clauses like ORDER BY. - int getRowLimit() const { return _rowLimit; } - - std::string resultFilePath() const; - std::string resultFileHttpUrl() const; - - /// Add the tasks defined in the UberJob to this UberJobData object. - void addTasks(std::vector> const& tasks) { - std::lock_guard tLg(_ujTasksMtx); - _ujTasks.insert(_ujTasks.end(), tasks.begin(), tasks.end()); - } - - /// Let the czar know the result is ready. - void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, - uint64_t headerCount); // TODO:UJ remove headerCount - - /// Let the Czar know there's been a problem. - bool responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); - - std::string getIdStr() const { return _idStr; } - std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } - - bool getCancelled() const { return _cancelled; } - - /// Cancel all Tasks in this UberJob. - void cancelAllTasks(); - /// Returns the LIMIT of rows for the query enforceable at the worker, where values <= 0 indicate /// that there is no limit to the number of rows sent back by the worker. /// Workers can only safely limit rows for queries that have the LIMIT clause without other related /// clauses like ORDER BY. int getRowLimit() { return _rowLimit; } + std::string buildUjResultFilePath(std::string const& resultsDirname); + std::string resultFilePath(); + std::string resultFileHttpUrl(); + private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, int czarPort, uint64_t queryId, int rowLimit, uint64_t maxTableSizeBytes, diff --git a/src/wconfig/WorkerConfig.h b/src/wconfig/WorkerConfig.h index bce19b6c86..117580558e 100644 --- a/src/wconfig/WorkerConfig.h +++ b/src/wconfig/WorkerConfig.h @@ -48,6 +48,56 @@ class AuthContext; // This header declarations namespace lsst::qserv::wconfig { +/// This class handles the special case for the configuration value representing +/// the communications protocol used which can have a text value of "HTTP" or " +/// "XROOTD", case-insenitive. +class ConfigValResultDeliveryProtocol : public util::ConfigVal { +public: + using CvrdpPtr = std::shared_ptr; + enum TEnum { + HTTP = 0, ///< Use HTTP protocol + XROOT = 1 ///< Use XROOTD file protocol + }; + + ConfigValResultDeliveryProtocol() = delete; + virtual ~ConfigValResultDeliveryProtocol() = default; + + static CvrdpPtr create(util::ConfigValMap& configValMap, std::string const& section, + std::string const& name, bool required, std::string const& defVal, + bool hidden = false) { + auto newPtr = CvrdpPtr(new ConfigValResultDeliveryProtocol(section, name, required, defVal, hidden)); + addToMapBase(configValMap, newPtr); + return newPtr; + } + + /// Return the appropriate TEnum for the given `str`, where "" returns HTTP. + /// @throws ConfigException + static TEnum parse(std::string const& str); + + /// Convert the TEnum `protocol` to the appropriate string. + static std::string toString(TEnum protocol); + + /// Return the string value of this object. + std::string getValStrDanger() const override { return toString(_val); } + + /// Return the string default value of this object. + std::string getDefValStrDanger() const override { return toString(_defVal); } + + void setValFromConfigStoreChild(util::ConfigStore const& configStore) override; + TEnum getVal() const { return _val; } + + void setVal(TEnum val) { + _val = val; + logValSet(); + } + +private: + ConfigValResultDeliveryProtocol(std::string const& section, std::string const& name, bool required, + std::string const& defVal, bool hidden) + : ConfigVal(section, name, required, hidden), _defVal(parse(defVal)), _val(_defVal) {} + TEnum const _defVal; ///< Default value for the item this class is storing. + TEnum _val; ///< Value for the item this class is storing. +}; /// Provide all configuration parameters for a Qserv worker instance. /// Parse an INI configuration file, identify required parameters and ignore @@ -385,6 +435,7 @@ class WorkerConfig { util::ConfigValTUInt::create(_configValMap, "czar", "DeadTimeSec", notReq, 180); CVTUIntPtr _czarComNumHttpThreads = util::ConfigValTUInt::create(_configValMap, "czar", "ComNumHttpThreads", notReq, 40); + }; } // namespace lsst::qserv::wconfig diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 2dd5bfc60b..0201283d13 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -128,70 +128,6 @@ struct Fixture { return jsJobMsgPtr; } - struct MsgInfo { - string const db = "LSST"; - string const table = "Object"; - string const qry = "SELECT AVG(yFlux_PS) from LSST.Object_3240"; - int const chunkId = 3240; - int const czarId = 5; - string const czarName = "cz5"; - string const czarHostName = "cz5host"; - int const czarPort = 3437; - string const targWorkerId = "a_worker"; - std::shared_ptr foreman; - int const queryId = 23; - int const jobId = 1; - int const uberJobId = 1; - int const attemptCount = 1; - int const scanRating = 1; - bool const scanInteractive = false; - int const maxTableSize = 5000; - bool const lockInMemory = false; - string const resultName = "resName"; - string const authKey = "noAuthKey"; - int const rowLimit = 0; - }; - - shared_ptr newTaskJson(MsgInfo const& mInfo) { - // Derived from TaskMsgFactory::makeMsgJson - - auto jsJobMsgPtr = std::shared_ptr( - new nlohmann::json({{"czarId", mInfo.czarId}, - {"queryId", mInfo.queryId}, - {"jobId", mInfo.jobId}, - {"attemptCount", mInfo.attemptCount}, - {"querySpecDb", mInfo.db}, - {"scanPriority", mInfo.scanRating}, - {"scanInteractive", mInfo.scanInteractive}, - {"maxTableSize", mInfo.maxTableSize}, - {"chunkScanTables", nlohmann::json::array()}, - {"chunkId", mInfo.chunkId}, - {"queryFragments", nlohmann::json::array()}})); - - auto& jsJobMsg = *jsJobMsgPtr; - - auto& chunkScanTables = jsJobMsg["chunkScanTables"]; - nlohmann::json cst = {{"db", mInfo.db}, - {"table", mInfo.table}, - {"lockInMemory", mInfo.lockInMemory}, - {"tblScanRating", mInfo.scanRating}}; - chunkScanTables.push_back(move(cst)); - - auto& jsFragments = jsJobMsg["queryFragments"]; - nlohmann::json jsFrag = {{"resultTable", mInfo.resultName}, - {"queries", nlohmann::json::array()}, - {"subchunkTables", nlohmann::json::array()}, - {"subchunkIds", nlohmann::json::array()}}; - - auto& jsQueries = jsFrag["queries"]; - nlohmann::json jsQry = {{"subQuery", mInfo.qry}}; - jsQueries.push_back(move(jsQry)); - - jsFragments.push_back(move(jsFrag)); - - return jsJobMsgPtr; - } - MySqlConfig newMySqlConfig() { string user = "qsmaster"; string password = ""; @@ -250,15 +186,17 @@ BOOST_AUTO_TEST_CASE(Simple) { shared_ptr crm = ChunkResourceMgr::newMgr(backend); SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); + auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); scanInfo->scanRating = mInfo.scanRating; scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, + mInfo.foreman, mInfo.authKey); + vector taskVect = - Task::createTasksForUnitTest(ujData, *msgJson, sChannel, scanInfo, mInfo.scanInteractive, - mInfo.maxTableSize, crm); + Task::createTasksForUnitTest(ujData, *msgJson, sChannel, mInfo.maxTableSize, crm); Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); @@ -301,15 +239,15 @@ BOOST_AUTO_TEST_CASE(Output) { shared_ptr crm = ChunkResourceMgr::newMgr(backend); SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); auto const queries = queriesAndChunks(); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.targWorkerId, mInfo.foreman, mInfo.authKey); auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); scanInfo->scanRating = mInfo.scanRating; scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - vector taskVect = - Task::createTasksForUnitTest(ujData, *msgJson, sc, scanInfo, mInfo.scanInteractive, - mInfo.maxTableSize, crm); + auto ujData = lsst::qserv::wbase::UberJobData::create( + mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, + mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, + mInfo.foreman, mInfo.authKey); + + vector taskVect = Task::createTasksForUnitTest(ujData, *msgJson, sc, mInfo.maxTableSize, crm); Task::Ptr task = taskVect[0]; QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); diff --git a/src/wpublish/QueriesAndChunks.cc b/src/wpublish/QueriesAndChunks.cc index 0b325fffc3..1410e819d2 100644 --- a/src/wpublish/QueriesAndChunks.cc +++ b/src/wpublish/QueriesAndChunks.cc @@ -143,7 +143,6 @@ void QueriesAndChunks::addTask(wbase::Task::Ptr const& task) { auto stats = addQueryId(qid, czId); stats->addTask(task); } - void QueriesAndChunks::addTasks(vector const& tasks, std::vector& cmds) { unique_lock guardStats(_queryStatsMapMtx); diff --git a/src/wsched/BlendScheduler.cc b/src/wsched/BlendScheduler.cc index 525b6afa9f..0d1ab2c0a9 100644 --- a/src/wsched/BlendScheduler.cc +++ b/src/wsched/BlendScheduler.cc @@ -270,7 +270,6 @@ void BlendScheduler::commandStart(util::Command::Ptr const& cmd) { LOGS(_log, LOG_LVL_ERROR, "BlendScheduler::commandStart scheduler not found"); } _infoChanged = true; - LOGS(_log, LOG_LVL_DEBUG, "BlendScheduler::commandStart &&& end"); } void BlendScheduler::commandFinish(util::Command::Ptr const& cmd) { diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index af565ad9c1..cafbfb8dc2 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -54,7 +54,7 @@ void ChunkTasksQueue::queueTask(std::vector const& tasks) { // Correct ChunkTask wasn't found, make a new one. std::pair ele(chunkId, std::make_shared(chunkId)); auto res = _chunkMap.insert(ele); // insert should fail if the key already exists. - LOGS(_log, LOG_LVL_DEBUG, " queueTask chunk=" << chunkId << " created=" << res.second); + LOGS(_log, LOG_LVL_TRACE, " queueTask chunk=" << chunkId << " created=" << res.second); iter = res.first; } } From 69dfdb6601ed4efa94937cf0dfd550082a010123 Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 12 Feb 2025 11:40:31 -0800 Subject: [PATCH 09/15] Added family map option to not use chunk size for distribution. --- deploy/docker/base/Dockerfile | 1 + src/cconfig/CMakeLists.txt | 8 +- src/cconfig/CzarConfig.cc | 15 -- src/ccontrol/CMakeLists.txt | 12 +- src/ccontrol/MergingHandler.cc | 63 +++--- src/ccontrol/MergingHandler.h | 30 +-- src/ccontrol/UserQuerySelect.cc | 27 ++- src/ccontrol/UserQuerySelect.h | 2 + src/czar/ActiveWorker.cc | 273 ------------------------- src/czar/CMakeLists.txt | 12 +- src/czar/Czar.cc | 10 - src/czar/CzarChunkMap.h | 2 - src/czar/CzarRegistry.cc | 12 -- src/czar/CzarRegistry.h | 22 -- src/czar/HttpCzarSvc.h | 4 - src/czar/HttpCzarWorkerModule.cc | 27 ++- src/czar/HttpCzarWorkerModule.h | 5 - src/czar/HttpSvc.h | 4 - src/czar/testCzar.cc | 4 +- src/global/CMakeLists.txt | 4 + src/mysql/CsvBuffer.cc | 4 + src/mysql/LocalInfile.cc | 3 + src/mysql/RowBuffer.h | 85 ++++++++ src/proto/CMakeLists.txt | 3 + src/protojson/CMakeLists.txt | 3 + src/protojson/JobReadyMsg.cc | 116 +++++++++++ src/protojson/JobReadyMsg.h | 99 +++++++++ src/protojson/UberJobMsg.cc | 18 -- src/protojson/UberJobMsg.h | 4 +- src/protojson/WorkerCzarComIssue.cc | 117 +++++++++++ src/protojson/WorkerCzarComIssue.h | 131 ++++++++++++ src/protojson/WorkerQueryStatusData.cc | 73 +------ src/protojson/WorkerQueryStatusData.h | 99 --------- src/protojson/testJobReadyMsg.cc | 106 ++++++++++ src/protojson/testStatusData.cc | 1 + src/proxy/CMakeLists.txt | 2 +- src/qdisp/CMakeLists.txt | 10 +- src/qdisp/Executive.cc | 138 ++----------- src/qdisp/Executive.h | 34 +-- src/qdisp/JobDescription.cc | 8 +- src/qdisp/JobDescription.h | 21 +- src/qdisp/JobQuery.cc | 20 +- src/qdisp/JobQuery.h | 17 -- src/qdisp/ResponseHandler.h | 21 -- src/qdisp/UberJob.cc | 41 ---- src/qdisp/testQDisp.cc | 130 +----------- src/qhttp/CMakeLists.txt | 2 + src/qmeta/CMakeLists.txt | 8 +- src/qmeta/QMeta.h | 16 -- src/qmeta/QMetaMysql.cc | 12 -- src/qmeta/QMetaMysql.h | 4 - src/qproc/CMakeLists.txt | 3 + src/replica/CMakeLists.txt | 6 +- src/rproc/CMakeLists.txt | 5 +- src/rproc/InfileMerger.cc | 35 +--- src/rproc/InfileMerger.h | 7 +- src/sql/CMakeLists.txt | 4 + src/util/Command.h | 4 +- src/util/Error.h | 12 +- src/util/InstanceCount.cc | 79 +++++-- src/util/InstanceCount.h | 21 +- src/util/MultiError.cc | 21 +- src/util/MultiError.h | 4 + src/util/ThreadPool.h | 4 +- src/wbase/CMakeLists.txt | 4 - src/wbase/FileChannelShared.cc | 20 +- src/wbase/FileChannelShared.h | 3 +- src/wbase/Task.cc | 21 +- src/wbase/Task.h | 27 ++- src/wbase/UberJobData.cc | 150 ++------------ src/wbase/UberJobData.h | 16 +- src/wbase/UserQueryInfo.h | 2 + src/wconfig/CMakeLists.txt | 3 - src/wconfig/WorkerConfig.h | 1 - src/wcontrol/CMakeLists.txt | 4 - src/wcontrol/WCzarInfoMap.cc | 1 + src/wdb/CMakeLists.txt | 11 +- src/wdb/QueryRunner.cc | 5 +- src/wdb/QueryRunner.h | 9 +- src/wpublish/CMakeLists.txt | 10 - src/wpublish/QueriesAndChunks.h | 7 - src/wpublish/QueryStatistics.h | 2 + src/wsched/CMakeLists.txt | 8 +- src/wsched/testSchedulers.cc | 1 - src/xrdsvc/CMakeLists.txt | 9 +- src/xrdsvc/HttpWorkerCzarModule.cc | 7 - 86 files changed, 1016 insertions(+), 1393 deletions(-) create mode 100644 src/mysql/RowBuffer.h create mode 100644 src/protojson/JobReadyMsg.cc create mode 100644 src/protojson/JobReadyMsg.h create mode 100644 src/protojson/WorkerCzarComIssue.cc create mode 100644 src/protojson/WorkerCzarComIssue.h create mode 100644 src/protojson/testJobReadyMsg.cc diff --git a/deploy/docker/base/Dockerfile b/deploy/docker/base/Dockerfile index 5df9aa40de..4e4f67e729 100644 --- a/deploy/docker/base/Dockerfile +++ b/deploy/docker/base/Dockerfile @@ -52,6 +52,7 @@ RUN dnf install -y 'dnf-command(config-manager)' \ protobuf-devel \ python3.12 \ python3.12-devel \ + jemalloc \ tree \ vim \ zip \ diff --git a/src/cconfig/CMakeLists.txt b/src/cconfig/CMakeLists.txt index 37a3f75816..1311a51863 100644 --- a/src/cconfig/CMakeLists.txt +++ b/src/cconfig/CMakeLists.txt @@ -4,13 +4,13 @@ target_sources(cconfig PRIVATE CzarConfig.cc ) -target_include_directories(cconfig PRIVATE - ${XROOTD_INCLUDE_DIRS} -) target_link_libraries(cconfig PUBLIC log - XrdSsiLib +) + +install( + TARGETS cconfig ) install( diff --git a/src/cconfig/CzarConfig.cc b/src/cconfig/CzarConfig.cc index bf2ede5639..9e26fb0aaa 100644 --- a/src/cconfig/CzarConfig.cc +++ b/src/cconfig/CzarConfig.cc @@ -28,7 +28,6 @@ #include // Third party headers -#include "XrdSsi/XrdSsiLogger.hh" // LSST headers #include "lsst/log/Log.h" @@ -43,20 +42,6 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.cconfig.CzarConfig"); -void QservLogger(struct timeval const& mtime, unsigned long tID, const char* msg, int mlen) { - static log4cxx::spi::LocationInfo xrdLoc( - "client", log4cxx::spi::LocationInfo::calcShortFileName("client"), "", 0); - static LOG_LOGGER myLog = LOG_GET("lsst.qserv.xrdssi.msgs"); - - if (myLog.isInfoEnabled()) { - while (mlen && msg[mlen - 1] == '\n') --mlen; // strip all trailing newlines - std::string theMsg(msg, mlen); - lsst::log::Log::MDC("LWP", std::to_string(tID)); - myLog.logMsg(log4cxx::Level::getInfo(), xrdLoc, theMsg); - } -} - -bool dummy = XrdSsiLogger::SetMCB(QservLogger, XrdSsiLogger::mcbClient); } // namespace namespace lsst::qserv::cconfig { diff --git a/src/ccontrol/CMakeLists.txt b/src/ccontrol/CMakeLists.txt index 71a2bd2e38..9d54d33c17 100644 --- a/src/ccontrol/CMakeLists.txt +++ b/src/ccontrol/CMakeLists.txt @@ -3,7 +3,6 @@ add_dependencies(ccontrol proto) target_include_directories(ccontrol PRIVATE ${ANTLR4_INCLUDE_DIR} - ${XROOTD_INCLUDE_DIRS} ) target_sources(ccontrol PRIVATE @@ -27,11 +26,15 @@ target_sources(ccontrol PRIVATE target_link_libraries(ccontrol PUBLIC boost_regex cconfig + css log parser - replica + proto sphgeom - XrdCl +) + +install( + TARGETS ccontrol ) install( @@ -65,3 +68,6 @@ ccontrol_tests( testCControl testUserQueryType ) + +# set_tests_properties(testCControl PROPERTIES WILL_FAIL 1) + diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 82f7e9fa26..8bd43e7381 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -34,7 +34,6 @@ // Third-party headers #include "curl/curl.h" -#include "XrdCl/XrdClFile.hh" // LSST headers #include "lsst/log/Log.h" @@ -127,7 +126,7 @@ std::tuple readHttpFileAndMergeHttp( uint32_t msgSizeBytes = 0; bool success = true; bool mergeHappened = false; - int headerCount = 0; + uint64_t headerCount = 0; uint64_t totalBytesRead = 0; try { auto exec = uberJob->getExecutive(); @@ -211,11 +210,9 @@ std::tuple readHttpFileAndMergeHttp( } // Parse and evaluate the message. - //&&&mergeHappened = messageIsReady(msgBuf.get(), msgSizeBytes, last); mergeHappened = true; bool messageReadyResult = messageIsReady(msgBuf.get(), msgSizeBytes, last); totalBytesRead += msgSizeBytes; - //&&&if (!mergeHappened) { if (!messageReadyResult) { success = false; throw runtime_error("message processing failed at offset " + @@ -265,10 +262,23 @@ std::tuple readHttpFileAndMergeHttp( namespace lsst::qserv::ccontrol { -MergingHandler::MergingHandler(std::shared_ptr merger, std::string const& tableName) - : _infileMerger{merger}, _tableName{tableName} {} +shared_ptr MergingHandler::_httpConnPool; +mutex MergingHandler::_httpConnPoolMutex; -MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_TRACE, __func__ << " " << _tableName); } +shared_ptr const& MergingHandler::_getHttpConnPool() { + lock_guard const lock(_httpConnPoolMutex); + if (nullptr == _httpConnPool) { + _httpConnPool = make_shared( + cconfig::CzarConfig::instance()->getResultMaxHttpConnections()); + } + return _httpConnPool; +} + +MergingHandler::MergingHandler(std::shared_ptr const& merger, + std::shared_ptr const& exec) + : _infileMerger(merger), _executive(exec) {} + +MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_TRACE, __func__); } bool MergingHandler::flush(proto::ResponseSummary const& resp) { @@ -316,35 +326,14 @@ bool MergingHandler::flush(proto::ResponseSummary const& resp) { } void MergingHandler::errorFlush(std::string const& msg, int code) { - _setError(code, msg); + _setError(code, msg, util::ErrorCode::RESULT_IMPORT); // Might want more info from result service. // Do something about the error. FIXME. LOGS(_log, LOG_LVL_ERROR, "Error receiving result."); } -// Note that generally we always have an _infileMerger object except during -// a unit test. I suppose we could try to figure out how to create one. -// -void MergingHandler::prepScrubResults(int jobId, int attemptCount) { - if (_infileMerger) _infileMerger->prepScrub(jobId, attemptCount); -} - std::ostream& MergingHandler::print(std::ostream& os) const { - return os << "MergingRequester(" << _tableName << ", flushed=" << (_flushed ? "true)" : "false)"); -} - -bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, - proto::ResponseData const& responseData) { - if (_flushed) { - throw util::Bug(ERR_LOC, "already flushed"); - } - bool const success = _infileMerger->mergeHttp(uberJob, responseData); - if (!success) { - LOGS(_log, LOG_LVL_WARN, __func__ << " failed"); - util::Error const& err = _infileMerger->getError(); - _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg()); - } - return success; + return os << "MergingRequester(flushed=" << (_flushed ? "true)" : "false)"); } bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, @@ -356,15 +345,16 @@ bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, if (!success) { LOGS(_log, LOG_LVL_WARN, __func__ << " failed"); util::Error const& err = _infileMerger->getError(); - _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg()); + _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg(), util::ErrorCode::RESULT_IMPORT); } return success; } -void MergingHandler::_setError(int code, std::string const& msg) { +void MergingHandler::_setError(int code, std::string const& msg, int errorState) { LOGS(_log, LOG_LVL_DEBUG, "_setError: code: " << code << ", message: " << msg); - std::lock_guard lock(_errorMutex); - _error = Error(code, msg); + auto exec = _executive.lock(); + if (exec == nullptr) return; + exec->addMultiError(code, msg, errorState); } tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expectedRows, @@ -414,10 +404,9 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe return {success, shouldCancel}; } -void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int status) { +void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int errState) { if (!_errorSet.exchange(true)) { - _error = util::Error(errorCode, errorMsg, util::ErrorCode::MYSQLEXEC); - _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); + _setError(errorCode, errorMsg, errState); } } diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 7de0e7be9f..7e33fe9cfd 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -39,6 +39,7 @@ class ResponseSummary; } // namespace lsst::qserv::proto namespace lsst::qserv::qdisp { +class Executive; class JobQuery; class UberJob; } // namespace lsst::qserv::qdisp @@ -64,20 +65,8 @@ class MergingHandler : public qdisp::ResponseHandler { virtual ~MergingHandler(); /// @param merger downstream merge acceptor - /// @param tableName target table for incoming data - MergingHandler(std::shared_ptr merger, std::string const& tableName); - - /// Process the response and read the result file if no error was reported by a worker. - /// @return true if successful (no error) - bool flush(proto::ResponseSummary const& resp) override; - - /// @see ResponseHandler::flushHttp - /// @see MerginHandler::_mergeHttp - std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, - uint64_t& resultRows) override; - - /// @see ResponseHandler::flushHttpError - void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; + MergingHandler(std::shared_ptr const& merger, + std::shared_ptr const& exec); /// @see ResponseHandler::flushHttp /// @see MerginHandler::_mergeHttp @@ -93,18 +82,12 @@ class MergingHandler : public qdisp::ResponseHandler { /// Print a string representation of the receiver to an ostream std::ostream& print(std::ostream& os) const override; - /// @return an error code and description - Error getError() const override { - std::lock_guard lock(_errorMutex); - return _error; - } - private: /// Call InfileMerger to do the work of merging this data to the result. bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); /// Set error code and string. - void _setError(int code, std::string const& msg); + void _setError(int code, std::string const& msg, int errorState); /// Check if the query is no longer active. /// This is used to prevent the query from being processed after it has been cancelled @@ -114,12 +97,11 @@ class MergingHandler : public qdisp::ResponseHandler { bool _queryIsNoLongerActive(std::shared_ptr const& jobQuery) const; std::shared_ptr _infileMerger; ///< Merging delegate - std::string _tableName; ///< Target table name - Error _error; ///< Error description std::atomic _errorSet{false}; ///< Set to true when an error is set. - mutable std::mutex _errorMutex; ///< Protect readers from partial updates bool _flushed{false}; ///< flushed to InfileMerger? std::string _wName{"~"}; ///< worker name + + std::weak_ptr _executive; ///< Weak pointer to the executive for errors. }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 44180fc4eb..8ba8e512ad 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -86,7 +86,8 @@ #include "global/LogContext.h" #include "proto/worker.pb.h" #include "qdisp/Executive.h" -#include "qdisp/MessageStore.h" +#include "qdisp/JobQuery.h" +#include "qmeta/MessageStore.h" #include "qmeta/QMeta.h" #include "qmeta/Exceptions.h" #include "qmeta/QMeta.h" @@ -284,7 +285,6 @@ void UserQuerySelect::submit() { cs = _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec, fillInChunkIdTag); chunks.push_back(cs->chunkId); } - std::string chunkResultName = _ttn->make(cs->chunkId); // This should only need to be set once as all jobs should have the same database name. if (cs->db != dbName) { @@ -299,9 +299,8 @@ void UserQuerySelect::submit() { ResourceUnit ru; ru.setAsDbChunk(cs->db, cs->chunkId); - qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( - _qMetaCzarId, exec->getId(), sequence, ru, - std::make_shared(_infileMerger, chunkResultName), cs, chunkResultName); + qdisp::JobDescription::Ptr jobDesc = + qdisp::JobDescription::create(_qMetaCzarId, exec->getId(), sequence, ru, cs); auto job = exec->add(jobDesc); ++sequence; } @@ -339,6 +338,14 @@ void UserQuerySelect::buildAndSendUberJobs() { return; } + if (exec->getCancelled() || exec->getSuperfluous()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive cancelled."); + } + + if (exec->getSuperfluous()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive superfluous, result already found."); + } + // Only one thread should be generating UberJobs for this user query at any given time. lock_guard fcLock(_buildUberJobMtx); LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << exec->getTotalJobs()); @@ -394,6 +401,7 @@ void UserQuerySelect::buildAndSendUberJobs() { map workerJobMap; vector missingChunks; + int attemptCountIncreased = 0; // unassignedChunksInQuery needs to be in numerical order so that UberJobs contain chunk numbers in // numerical order. The workers run shared scans in numerical order of chunkId numbers. // Numerical order keeps the number of partially complete UberJobs running on a worker to a minimum, @@ -401,6 +409,7 @@ void UserQuerySelect::buildAndSendUberJobs() { for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { bool const increaseAttemptCount = true; jqPtr->getDescription()->incrAttemptCount(exec, increaseAttemptCount); + attemptCountIncreased++; // If too many workers are down, there will be a chunk that cannot be found. // Just continuing should leave jobs `unassigned` with their attempt count @@ -462,7 +471,8 @@ void UserQuerySelect::buildAndSendUberJobs() { if (wInfUJ->uberJobPtr == nullptr) { auto ujId = _uberJobIdSeq++; // keep ujId consistent string uberResultName = _ttn->make(ujId); - auto respHandler = make_shared(_infileMerger, uberResultName); + auto respHandler = + ccontrol::MergingHandler::Ptr(new ccontrol::MergingHandler(_infileMerger, exec)); auto uJob = qdisp::UberJob::create(exec, respHandler, exec->getId(), ujId, _qMetaCzarId, targetWorker); uJob->setWorkerContactInfo(wInfUJ->wInf); @@ -489,6 +499,11 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_ERROR, errStr); } + if (attemptCountIncreased > 0) { + LOGS(_log, LOG_LVL_WARN, + funcN << " increased attempt count for " << attemptCountIncreased << " Jobs"); + } + // Queue unqued UberJobs, these have less than the max number of jobs. for (auto const& [wIdKey, winfUjPtr] : workerJobMap) { if (winfUjPtr != nullptr) { diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index 6447f8fcd5..c744b18997 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -95,6 +95,8 @@ class UserQuerySelect : public UserQuery { UserQuerySelect(UserQuerySelect const&) = delete; UserQuerySelect& operator=(UserQuerySelect const&) = delete; + ~UserQuerySelect() override = default; + /** * @param resultLocation: Result location, if empty use result table with unique * name generated from query ID. diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index db59c6720f..81d011532c 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -236,232 +236,6 @@ string ActiveWorker::_dump() const { return os.str(); } -string ActiveWorker::getStateStr(State st) { - switch (st) { - case ALIVE: - return string("ALIVE"); - case QUESTIONABLE: - return string("QUESTIONABLE"); - case DEAD: - return string("DEAD"); - } - return string("unknown"); -} - -bool ActiveWorker::compareContactInfo(http::WorkerContactInfo const& wcInfo) const { - lock_guard lg(_aMtx); - auto wInfo_ = _wqsData->getWInfo(); - if (wInfo_ == nullptr) return false; - return wInfo_->isSameContactInfo(wcInfo); -} - -void ActiveWorker::setWorkerContactInfo(protojson::WorkerContactInfo::Ptr const& wcInfo) { - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " new info=" << wcInfo->dump()); - lock_guard lg(_aMtx); - _wqsData->setWInfo(wcInfo); -} - -void ActiveWorker::_changeStateTo(State newState, double secsSinceUpdate, string const& note) { - auto lLvl = (newState == DEAD) ? LOG_LVL_ERROR : LOG_LVL_INFO; - LOGS(_log, lLvl, - note << " oldState=" << getStateStr(_state) << " newState=" << getStateStr(newState) - << " secsSince=" << secsSinceUpdate); - _state = newState; -} - -void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double timeoutDeadSecs, - double maxLifetime) { - LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " start"); - bool newlyDeadWorker = false; - protojson::WorkerContactInfo::Ptr wInfo_; - { - lock_guard lg(_aMtx); - wInfo_ = _wqsData->getWInfo(); - if (wInfo_ == nullptr) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no WorkerContactInfo"); - return; - } - double secsSinceUpdate = wInfo_->timeSinceRegUpdateSeconds(); - LOGS(_log, LOG_LVL_TRACE, - cName(__func__) << " wInfo=" << wInfo_->dump() - << " secsSince=" << wInfo_->timeSinceRegUpdateSeconds() - << " secsSinceUpdate=" << secsSinceUpdate); - - // Update the last time the registry contacted this worker. - // TODO:UJ - This needs to be added to the dashboard. - switch (_state) { - case ALIVE: { - if (secsSinceUpdate >= timeoutAliveSecs) { - _changeStateTo(QUESTIONABLE, secsSinceUpdate, cName(__func__)); - } - break; - } - case QUESTIONABLE: { - if (secsSinceUpdate < timeoutAliveSecs) { - _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); - } - if (secsSinceUpdate >= timeoutDeadSecs) { - _changeStateTo(DEAD, secsSinceUpdate, cName(__func__)); - // All uberjobs for this worker need to die. - newlyDeadWorker = true; - } - break; - } - case DEAD: { - if (secsSinceUpdate < timeoutAliveSecs) { - _changeStateTo(ALIVE, secsSinceUpdate, cName(__func__)); - } else { - // Don't waste time on this worker until the registry has heard from it. - return; - } - break; - } - } - } - - // _aMtx must not be held when calling this. - if (newlyDeadWorker) { - LOGS(_log, LOG_LVL_WARN, - cName(__func__) << " worker " << wInfo_->wId << " appears to have died, reassigning its jobs."); - czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInfo_->wId); - } - - shared_ptr jsWorkerReqPtr; - { - // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a - // message to send to the worker. - jsWorkerReqPtr = _wqsData->serializeJson(maxLifetime); - } - - // Always send the message as it's a way to inform the worker that this - // czar is functioning and capable of receiving requests. - Ptr thisPtr = shared_from_this(); - auto sendStatusMsgFunc = [thisPtr, wInfo_, jsWorkerReqPtr](util::CmdData*) { - thisPtr->_sendStatusMsg(wInfo_, jsWorkerReqPtr); - }; - - auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(sendStatusMsgFunc)); - auto qdisppool = czar::Czar::getCzar()->getQdispPool(); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " queuing message"); - qdisppool->queCmd(cmd, 1); -} - -void ActiveWorker::_sendStatusMsg(protojson::WorkerContactInfo::Ptr const& wInf, - std::shared_ptr const& jsWorkerReqPtr) { - auto& jsWorkerReq = *jsWorkerReqPtr; - auto const method = http::Method::POST; - if (wInf == nullptr) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo was null."); - return; - } - auto [ciwId, ciwHost, ciwManag, ciwPort] = wInf->getAll(); - string const url = "http://" + ciwHost + ":" + to_string(ciwPort) + "/querystatus"; - vector const headers = {"Content-Type: application/json"}; - auto const& czarConfig = cconfig::CzarConfig::instance(); - - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " REQ " << jsWorkerReq); - string const requestContext = "Czar: '" + http::method2string(method) + "' stat request to '" + url + "'"; - LOGS(_log, LOG_LVL_TRACE, - cName(__func__) << " czarPost url=" << url << " request=" << jsWorkerReq.dump() - << " headers=" << headers[0]); - http::Client client(method, url, jsWorkerReq.dump(), headers); - bool transmitSuccess = false; - string exceptionWhat; - json response; - try { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read start"); - response = client.readAsJson(); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " read end"); - if (0 != response.at("success").get()) { - bool startupTimeChanged = false; - startupTimeChanged = _wqsData->handleResponseJson(response); - transmitSuccess = true; - if (startupTimeChanged) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " worker startupTime changed, likely rebooted."); - // kill all incomplete UberJobs on this worker. - czar::Czar::getCzar()->killIncompleteUbjerJobsOn(wInf->wId); - } - } else { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure response success=0 " << response); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_ERROR, requestContext + " transmit failure, ex: " + ex.what()); - exceptionWhat = ex.what(); - } - if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " transmit failure " << jsWorkerReq.dump() << " resp=" << response); - } -} - -void ActiveWorker::addToDoneDeleteFiles(QueryId qId) { _wqsData->addToDoneDeleteFiles(qId); } - -void ActiveWorker::addToDoneKeepFiles(QueryId qId) { _wqsData->addToDoneKeepFiles(qId); } - -void ActiveWorker::removeDeadUberJobsFor(QueryId qId) { _wqsData->removeDeadUberJobsFor(qId); } - -void ActiveWorker::addDeadUberJob(QueryId qId, UberJobId ujId) { - auto now = CLOCK::now(); - _wqsData->addDeadUberJob(qId, ujId, now); -} - -protojson::WorkerContactInfo::Ptr ActiveWorker::getWInfo() const { - std::lock_guard lg(_aMtx); - if (_wqsData == nullptr) return nullptr; - return _wqsData->getWInfo(); -} - -ActiveWorker::State ActiveWorker::getState() const { - std::lock_guard lg(_aMtx); - return _state; -} - -string ActiveWorker::dump() const { - lock_guard lg(_aMtx); - return _dump(); -} - -string ActiveWorker::_dump() const { - stringstream os; - os << "ActiveWorker " << (_wqsData->dump()); - return os.str(); -} - -ActiveWorkerMap::ActiveWorkerMap(std::shared_ptr const& czarConfig) - : _timeoutAliveSecs(czarConfig->getActiveWorkerTimeoutAliveSecs()), - _timeoutDeadSecs(czarConfig->getActiveWorkerTimeoutDeadSecs()), - _maxLifetime(czarConfig->getActiveWorkerMaxLifetimeSecs()) {} - -void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap, - protojson::CzarContactInfo::Ptr const& czInfo, - std::string const& replicationInstanceId, - std::string const& replicationAuthKey) { - // Go through wcMap, update existing entries in _awMap, create new entries for those that don't exist, - lock_guard awLg(_awMapMtx); - for (auto const& [wcKey, wcVal] : wcMap) { - auto iter = _awMap.find(wcKey); - if (iter == _awMap.end()) { - auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " AciveWorker created for " << wcKey); - _awMap[wcKey] = newAW; - if (_czarCancelAfterRestart) { - newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); - } - } else { - auto aWorker = iter->second; - if (!aWorker->compareContactInfo(*wcVal)) { - // This should not happen, but try to handle it gracefully if it does. - LOGS(_log, LOG_LVL_WARN, - cName(__func__) << " worker contact info changed for " << wcKey - << " new=" << wcVal->dump() << " old=" << aWorker->dump()); - // If there is existing information, only host and port values will change. - aWorker->setWorkerContactInfo(wcVal); - } - aWorker->getWInfo()->setRegUpdateTime(wcVal->getRegUpdateTime()); - } - } -} - void ActiveWorkerMap::setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { _czarCancelAfterRestart = true; _czarCancelAfterRestartCzId = czId; @@ -499,16 +273,6 @@ void ActiveWorkerMap::addToDoneKeepFiles(QueryId qId) { } } -/* &&& -/// &&& doc -void ActiveWorkerMap::removeDeadUberJobsFor(QueryId qId) { - lock_guard lck(_awMapMtx); - for (auto const& [wName, awPtr] : _awMap) { - awPtr->removeDeadUberJobsFor(qId); - } -} -*/ - ActiveWorkerMap::ActiveWorkerMap(std::shared_ptr const& czarConfig) : _timeoutAliveSecs(czarConfig->getActiveWorkerTimeoutAliveSecs()), _timeoutDeadSecs(czarConfig->getActiveWorkerTimeoutDeadSecs()), @@ -544,41 +308,4 @@ void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap } } -void ActiveWorkerMap::setCzarCancelAfterRestart(CzarIdType czId, QueryId lastQId) { - _czarCancelAfterRestart = true; - _czarCancelAfterRestartCzId = czId; - _czarCancelAfterRestartQId = lastQId; -} - -ActiveWorker::Ptr ActiveWorkerMap::getActiveWorker(string const& workerId) const { - lock_guard lck(_awMapMtx); - auto iter = _awMap.find(workerId); - if (iter == _awMap.end()) return nullptr; - return iter->second; -} - -void ActiveWorkerMap::sendActiveWorkersMessages() { - // Send messages to each active worker as needed - lock_guard lck(_awMapMtx); - for (auto&& [wName, awPtr] : _awMap) { - awPtr->updateStateAndSendMessages(_timeoutAliveSecs, _timeoutDeadSecs, _maxLifetime); - } -} - -void ActiveWorkerMap::addToDoneDeleteFiles(QueryId qId) { - lock_guard lck(_awMapMtx); - for (auto const& [wName, awPtr] : _awMap) { - awPtr->addToDoneDeleteFiles(qId); - awPtr->removeDeadUberJobsFor(qId); - } -} - -void ActiveWorkerMap::addToDoneKeepFiles(QueryId qId) { - lock_guard lck(_awMapMtx); - for (auto const& [wName, awPtr] : _awMap) { - awPtr->addToDoneKeepFiles(qId); - awPtr->removeDeadUberJobsFor(qId); - } -} - } // namespace lsst::qserv::czar diff --git a/src/czar/CMakeLists.txt b/src/czar/CMakeLists.txt index e4025b79e9..1257769cc6 100644 --- a/src/czar/CMakeLists.txt +++ b/src/czar/CMakeLists.txt @@ -1,4 +1,5 @@ add_library(czar SHARED) +add_dependencies(czar proto) target_sources(czar PRIVATE ActiveWorker.cc @@ -20,9 +21,6 @@ target_sources(czar PRIVATE WorkerIngestProcessor.cc ) -target_include_directories(czar PRIVATE - ${XROOTD_INCLUDE_DIRS} -) target_link_libraries(czar PUBLIC cconfig @@ -32,20 +30,18 @@ target_link_libraries(czar PUBLIC qhttp util log - XrdSsiLib cpp-httplib boost_program_options ) install( - TARGETS czar -) + TARGETS czar +) function(CZAR_UTILS) foreach(UTIL IN ITEMS ${ARGV}) add_executable(${UTIL}) target_sources(${UTIL} PRIVATE ${UTIL}.cc) - target_include_directories(${UTIL} PRIVATE ${XROOTD_INCLUDE_DIRS}) target_link_libraries(${UTIL} PRIVATE cconfig ccontrol @@ -82,7 +78,7 @@ function(czar_tests) qana qdisp qproc - qserv_meta + qmeta query rproc sql diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 05735f898d..3a5a074306 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -113,9 +113,6 @@ void Czar::_monitor() { // check if workers have died by timeout. _czarRegistry->sendActiveWorkersMessages(); - // &&& Send appropriate messages to all ActiveWorkers - _czarRegistry->sendActiveWorkersMessages(); - /// Create new UberJobs (if possible) for all jobs that are /// unassigned for any reason. map> execMap; @@ -261,13 +258,6 @@ Czar::~Czar() { LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar() end"); } -Czar::~Czar() { - LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar()"); - _monitorLoop = false; - _monitorThrd.join(); - LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar() end"); -} - SubmitResult Czar::submitQuery(string const& query, map const& hints) { LOGS(_log, LOG_LVL_DEBUG, "New query: " << query << ", hints: " << util::printable(hints)); diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index 8188dfa697..a167ba0a2c 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -94,8 +94,6 @@ class CzarChunkMap { ~CzarChunkMap(); - ~CzarChunkMap(); - class WorkerChunksData; /// Essentially a structure for storing data about which tables and workers are associated with this diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index c0cb013715..e81b0e168c 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -235,16 +235,4 @@ void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) _activeWorkerMap->sendActiveWorkersMessages(); } -void CzarRegistry::endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults) { - // Add query id to the appropriate list. - if (deleteWorkerResults) { - _activeWorkerMap->addToDoneDeleteFiles(qId); - } else { - _activeWorkerMap->addToDoneKeepFiles(qId); - } - - // With lists updated, send out messages. - _activeWorkerMap->sendActiveWorkersMessages(); -} - } // namespace lsst::qserv::czar diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index 57d1312dd1..08d24a7bcc 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -88,28 +88,6 @@ class CzarRegistry { /// `deleteWorkerResults` is true. void endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults); - /// Return _contactMap, the object that the returned pointer points to is - /// constant and no attempts should be made to change it. - http::WorkerContactInfo::WCMapPtr getWorkerContactMap() { - std::lock_guard lockG(_mapMtx); - return _contactMap; - } - - /// Return _contactMap, the object that the returned pointer points to is - /// constant and no attempts should be made to change it. This - /// function will wait forever for a valid contact map to be ready. - protojson::WorkerContactInfo::WCMapPtr waitForWorkerContactMap() const; - - /// Send all live workers the `WorkerQueryStatusData` message for - /// that worker. This may result in the worker sending back the - /// `WorkerCzarComIssue` message if there were communication problems. - void sendActiveWorkersMessages(); - - /// Add the query id to the list of queries to end on workers and - /// send the messages, deleting all result files if - /// `deleteWorkerResults` is true. - void endUserQueryOnWorkers(QueryId qId, bool deleteWorkerResults); - private: CzarRegistry() = delete; CzarRegistry(std::shared_ptr const& czarConfig, diff --git a/src/czar/HttpCzarSvc.h b/src/czar/HttpCzarSvc.h index 2b984fbdd0..b3cb32c743 100644 --- a/src/czar/HttpCzarSvc.h +++ b/src/czar/HttpCzarSvc.h @@ -43,10 +43,6 @@ namespace httplib { class SSLServer; } // namespace httplib -namespace lsst::qserv::wcontrol { -class Foreman; -} // namespace lsst::qserv::wcontrol - // This header declarations namespace lsst::qserv::czar { diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 9d134aead5..f009dbbae7 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -29,6 +29,8 @@ // Qserv headers #include "cconfig/CzarConfig.h" #include "czar/Czar.h" +#include "protojson/JobReadyMsg.h" +#include "protojson/WorkerCzarComIssue.h" #include "qdisp/Executive.h" #include "qdisp/UberJob.h" #include "global/intTypes.h" @@ -146,19 +148,15 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { // Parse and verify the json message and then have the uberjob import the file. json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // TODO:UJ file response - move construction and parsing - // TODO:UJ to a class so it can be added to WorkerCzarComIssue - // See wbase::UberJobData::responseFileReady - string const targetWorkerId = body().required("workerid"); - string const czarName = body().required("czar"); - qmeta::CzarId const czarId = body().required("czarid"); - QueryId const queryId = body().required("queryid"); - UberJobId const uberJobId = body().required("uberjobid"); - string const fileUrl = body().required("fileUrl"); - uint64_t const rowCount = body().required("rowCount"); - uint64_t const fileSize = body().required("fileSize"); + string const repliInstanceId = cconfig::CzarConfig::instance()->replicationInstanceId(); + string const repliAuthKey = cconfig::CzarConfig::instance()->replicationAuthKey(); + auto const& jsReq = body().objJson; + auto jrMsg = protojson::JobReadyMsg::createFromJson(jsReq, repliInstanceId, repliAuthKey); // Find UberJob + auto queryId = jrMsg->getQueryId(); + auto czarId = jrMsg->getCzarId(); + auto uberJobId = jrMsg->getUberJobId(); qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); if (exec == nullptr) { throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + @@ -172,10 +170,11 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { " czar=" + to_string(czarId)); } - uj->setResultFileSize(fileSize); - exec->checkResultFileSize(fileSize); + uj->setResultFileSize(jrMsg->getFileSize()); + exec->checkResultFileSize(jrMsg->getFileSize()); - auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); + auto importRes = + uj->importResultFile(jrMsg->getFileUrl(), jrMsg->getRowCount(), jrMsg->getFileSize()); jsRet = importRes; } catch (std::invalid_argument const& iaEx) { diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h index e849172b6c..a6d21536c0 100644 --- a/src/czar/HttpCzarWorkerModule.h +++ b/src/czar/HttpCzarWorkerModule.h @@ -81,11 +81,6 @@ class HttpCzarWorkerModule : public QhttpModule { /// Translates the issues and calls the Czar to take action. nlohmann::json _handleWorkerCzarComIssue(std::string const& func); - /// &&& doc - nlohmann::json _handleJobReady(std::string const& func); - - /// Translates the error and calls the Czar to take action. - nlohmann::json _handleJobError(std::string const& func); }; } // namespace lsst::qserv::czar diff --git a/src/czar/HttpSvc.h b/src/czar/HttpSvc.h index 8fd75a25f8..36d346978f 100644 --- a/src/czar/HttpSvc.h +++ b/src/czar/HttpSvc.h @@ -35,10 +35,6 @@ namespace lsst::qserv::qhttp { class Server; } // namespace lsst::qserv::qhttp -namespace lsst::qserv::wcontrol { -class Foreman; -} // namespace lsst::qserv::wcontrol - // This header declarations namespace lsst::qserv::czar { diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index aad9fdfd31..45f42f8318 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -191,13 +191,13 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { auto jsTest1 = nlohmann::json::parse(test1); qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); - auto familyMap = czFamMap.makeNewMaps(qChunkMap1); + auto familyMap = czFamMap.makeNewMaps(qChunkMap1, true); czar::CzarFamilyMap::verify(familyMap); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 1 passed"); auto jsTest2 = nlohmann::json::parse(test2); qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); - auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2); + auto familyMap2 = czFamMap.makeNewMaps(qChunkMap2, true); czar::CzarFamilyMap::verify(familyMap2); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 2 passed"); } diff --git a/src/global/CMakeLists.txt b/src/global/CMakeLists.txt index 25e94c496d..a8fe4e782b 100644 --- a/src/global/CMakeLists.txt +++ b/src/global/CMakeLists.txt @@ -8,6 +8,10 @@ target_sources(global PRIVATE stringUtil.cc ) +install( + TARGETS global +) + target_link_libraries(global PUBLIC log ) diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index bf1ab253bb..fcd58372df 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -33,11 +33,15 @@ // Third-party headers #include +// LSST headers +#include "lsst/log/Log.h" + // Qserv headers #include "mysql/LocalInfileError.h" #include "mysql/MySqlUtils.h" namespace { + std::string const mysqlNull("\\N"); int const largeRowThreshold = 500 * 1024; // should be less than 0.5 * infileBufferSize diff --git a/src/mysql/LocalInfile.cc b/src/mysql/LocalInfile.cc index b073654972..dc3a26484d 100644 --- a/src/mysql/LocalInfile.cc +++ b/src/mysql/LocalInfile.cc @@ -77,6 +77,7 @@ LocalInfile::LocalInfile(char const* filename, std::shared_ptr csvBuf } LocalInfile::~LocalInfile() { + LOGS(_log, LOG_LVL_TRACE, "~LocalInfile"); if (_buffer) { delete[] _buffer; } @@ -131,6 +132,8 @@ int LocalInfile::getError(char* buf, unsigned int bufLen) { return 0; } +LocalInfile::Mgr::~Mgr() { LOGS(_log, LOG_LVL_TRACE, "LocalInfile::Mgr::~Mgr()"); } + void LocalInfile::Mgr::attach(MYSQL* mysql) { mysql_set_local_infile_handler(mysql, local_infile_init, local_infile_read, local_infile_end, local_infile_error, this); diff --git a/src/mysql/RowBuffer.h b/src/mysql/RowBuffer.h new file mode 100644 index 0000000000..389790da9c --- /dev/null +++ b/src/mysql/RowBuffer.h @@ -0,0 +1,85 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2015 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +#ifndef LSST_QSERV_MYSQL_ROWBUFFER_H +#define LSST_QSERV_MYSQL_ROWBUFFER_H + +// System headers +#include +#include +#include + +// Third-party headers +#include + +namespace lsst::qserv::proto { +class Result; +} // namespace lsst::qserv::proto + +namespace lsst::qserv::mysql { + +/// Row is a mysql row abstraction that bundles field sizes and counts. Row is +/// shallow, and does not perform any memory management. +struct Row { + Row() : row(nullptr), lengths(nullptr), numFields(-1) {} + virtual ~Row(); + + // Shallow copies all-around. + Row(char** row_, unsigned long int* lengths_, int numFields_) + : row(row_), lengths(lengths_), numFields(numFields_) {} + + unsigned int minRowSize() const { + unsigned int sum = 0; + for (int i = 0; i < numFields; ++i) { + sum += lengths[i]; + } + return sum; + } + + char** row; + unsigned long int* lengths; + int numFields; +}; + +/// RowBuffer: an buffer from which arbitrarily-sized buckets of bytes +/// can be read. The buffer represents a tab-separated-field, +/// line-delimited-tuple sequence of tuples. +class RowBuffer { +public: + typedef std::shared_ptr Ptr; + + virtual ~RowBuffer(); + + /// Fetch a number of bytes into a buffer. Return the number of bytes + /// fetched. Returning less than bufLen does NOT indicate EOF. + virtual unsigned fetch(char* buffer, unsigned bufLen) = 0; + + /// Construct a RowBuffer tied to a MySQL query result + static Ptr newResRowBuffer(MYSQL_RES* result); + + /// Return a descriptive string. + virtual std::string dump() const = 0; +}; + +} // namespace lsst::qserv::mysql +#endif // LSST_QSERV_MYSQL_ROWBUFFER_H diff --git a/src/proto/CMakeLists.txt b/src/proto/CMakeLists.txt index e86e15216c..2e6e55ab12 100644 --- a/src/proto/CMakeLists.txt +++ b/src/proto/CMakeLists.txt @@ -18,3 +18,6 @@ target_link_libraries(proto PUBLIC protobuf ) +install( + TARGETS proto +) diff --git a/src/protojson/CMakeLists.txt b/src/protojson/CMakeLists.txt index 8ac88b4cda..35e74c708d 100644 --- a/src/protojson/CMakeLists.txt +++ b/src/protojson/CMakeLists.txt @@ -1,8 +1,10 @@ add_library(protojson SHARED) target_sources(protojson PRIVATE + JobReadyMsg.cc ScanTableInfo.cc UberJobMsg.cc + WorkerCzarComIssue.cc WorkerQueryStatusData.cc ) @@ -35,6 +37,7 @@ function(PROTOJSON_TESTS) endfunction() protojson_tests( + testJobReadyMsg testStatusData testUberJobMsg ) diff --git a/src/protojson/JobReadyMsg.cc b/src/protojson/JobReadyMsg.cc new file mode 100644 index 0000000000..cccd230cbb --- /dev/null +++ b/src/protojson/JobReadyMsg.cc @@ -0,0 +1,116 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "protojson/JobReadyMsg.h" + +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "util/common.h" +#include "util/TimeUtils.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.JobReadyMsg"); +} // namespace + +namespace lsst::qserv::protojson { + +JobReadyMsg::Ptr JobReadyMsg::create(std::string const& replicationInstanceId, + std::string const& replicationAuthKey, std::string const& workerIdStr, + std::string const& czarName, CzarIdType czarId, QueryId queryId, + UberJobId uberJobId, std::string const& fileUrl, uint64_t rowCount, + uint64_t fileSize) { + auto jrMsg = Ptr(new JobReadyMsg(replicationInstanceId, replicationAuthKey)); + jrMsg->_workerId = workerIdStr; + jrMsg->_czarName = czarName; + jrMsg->_czarId = czarId; + jrMsg->_queryId = queryId; + jrMsg->_uberJobId = uberJobId; + jrMsg->_fileUrl = fileUrl; + jrMsg->_rowCount = rowCount; + jrMsg->_fileSize = fileSize; + return jrMsg; +} + +JobReadyMsg::Ptr JobReadyMsg::createFromJson(nlohmann::json const& jsWReq, + std::string const& replicationInstanceId, + std::string const& replicationAuthKey) { + string const fName("JobReadyMsg::createFromJson"); + LOGS(_log, LOG_LVL_DEBUG, fName); + try { + if (jsWReq["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, fName << " bad version"); + return nullptr; + } + + // Presumably, if these were wrong, it wouldn't have gotten this far. + auto repliInstId = http::RequestBodyJSON::required(jsWReq, "instance_id"); + auto repliAuthKey = http::RequestBodyJSON::required(jsWReq, "auth_key"); + + auto jrMsg = create(repliInstId, repliAuthKey); + + jrMsg->_workerId = http::RequestBodyJSON::required(jsWReq, "workerid"); + jrMsg->_czarName = http::RequestBodyJSON::required(jsWReq, "czar"); + jrMsg->_czarId = http::RequestBodyJSON::required(jsWReq, "czarid"); + jrMsg->_queryId = http::RequestBodyJSON::required(jsWReq, "queryid"); + jrMsg->_uberJobId = http::RequestBodyJSON::required(jsWReq, "uberjobid"); + jrMsg->_fileUrl = http::RequestBodyJSON::required(jsWReq, "fileUrl"); + jrMsg->_rowCount = http::RequestBodyJSON::required(jsWReq, "rowCount"); + jrMsg->_fileSize = http::RequestBodyJSON::required(jsWReq, "fileSize"); + return jrMsg; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("JobReadyMsg::createJson invalid ") << exc.what()); + } + return nullptr; +} + +json JobReadyMsg::serializeJson() { + shared_ptr jsJrReqPtr = make_shared(); + json& jsJr = *jsJrReqPtr; + + // These need to match what http::BaseModule::enforceInstanceId() + // and http::BaseModule::enforceAuthorization() are looking for. + jsJr["instance_id"] = _replicationInstanceId; + jsJr["auth_key"] = _replicationAuthKey; + + jsJr["version"] = http::MetaModule::version; + jsJr["workerid"] = _workerId; + jsJr["czar"] = _czarName; + jsJr["czarid"] = _czarId; + jsJr["queryid"] = _queryId; + jsJr["uberjobid"] = _uberJobId; + jsJr["fileUrl"] = _fileUrl; + jsJr["rowCount"] = _rowCount; + jsJr["fileSize"] = _fileSize; + return jsJr; +} + +} // namespace lsst::qserv::protojson diff --git a/src/protojson/JobReadyMsg.h b/src/protojson/JobReadyMsg.h new file mode 100644 index 0000000000..4105421256 --- /dev/null +++ b/src/protojson/JobReadyMsg.h @@ -0,0 +1,99 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_PROTOJSON_JOBREADYMSG_H +#define LSST_QSERV_PROTOJSON_JOBREADYMSG_H + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "global/clock_defs.h" +#include "global/intTypes.h" +#include "protojson/WorkerQueryStatusData.h" + +// This header declarations +namespace lsst::qserv::protojson { + +/// This class handles the message used to inform the czar that a result file +/// for an UberJob is ready. +class JobReadyMsg { +public: + using Ptr = std::shared_ptr; + + JobReadyMsg() = delete; + JobReadyMsg(std::string const& replicationInstanceId, std::string const& replicationAuthKey) + : _replicationInstanceId(replicationInstanceId), _replicationAuthKey(replicationAuthKey) {} + JobReadyMsg(JobReadyMsg const&) = delete; + JobReadyMsg& operator=(JobReadyMsg const&) = delete; + + std::string cName(const char* fName) { return std::string("WorkerQueryStatusData::") + fName; } + + static Ptr create(std::string const& replicationInstanceId, std::string const& replicationAuthKey, + std::string const& workerIdStr, std::string const& czarName, CzarIdType czarId, + QueryId queryId, UberJobId uberJobId, std::string const& fileUrl, uint64_t rowCount, + uint64_t fileSize); + + static Ptr create(std::string const& replicationInstanceId, std::string const& replicationAuthKey) { + return Ptr(new JobReadyMsg(replicationInstanceId, replicationAuthKey)); + } + + /// This function creates a JobReadyMsg object from the worker json `czarJson`, the + /// other parameters are used to verify the json message. + static Ptr createFromJson(nlohmann::json const& czarJson, std::string const& replicationInstanceId, + std::string const& replicationAuthKey); + + ~JobReadyMsg() = default; + + /// Return a json object with data allowing collection of UberJob result file. + nlohmann::json serializeJson(); + + std::string getWorkerId() const { return _workerId; } + std::string getCzarName() const { return _czarName; } + CzarIdType getCzarId() const { return _czarId; } + QueryId getQueryId() const { return _queryId; } + UberJobId getUberJobId() const { return _uberJobId; } + std::string getFileUrl() const { return _fileUrl; } + uint64_t getRowCount() const { return _rowCount; } + uint64_t getFileSize() const { return _fileSize; } + +private: + std::string const _replicationInstanceId; + std::string const _replicationAuthKey; + std::string _workerId; + std::string _czarName; + CzarIdType _czarId = 0; + QueryId _queryId = 0; + UberJobId _uberJobId = 0; + std::string _fileUrl; + uint64_t _rowCount = 0; + uint64_t _fileSize = 0; +}; + +} // namespace lsst::qserv::protojson + +#endif // LSST_QSERV_PROTOJSON_JOBREADYMSG_H diff --git a/src/protojson/UberJobMsg.cc b/src/protojson/UberJobMsg.cc index 312f138a6e..e95560de09 100644 --- a/src/protojson/UberJobMsg.cc +++ b/src/protojson/UberJobMsg.cc @@ -448,24 +448,6 @@ string JobFragment::dump() const { return os.str(); } -string JobFragment::dump() const { - stringstream os; - os << " templateIndexes={"; - for (int j : _jobSubQueryTempIndexes) { - os << j << ", "; - } - os << "} subchunkIds={"; - for (int j : _subchunkIds) { - os << j << ", "; - } - os << "} dbtbl={"; - for (int j : _subchunkIds) { - os << j << ", "; - } - os << "}"; - return os.str(); -} - nlohmann::json JobFragment::serializeJson() const { json jsFragment = {{"subquerytemplate_indexes", _jobSubQueryTempIndexes}, {"dbtables_indexes", _jobDbTablesIndexes}, diff --git a/src/protojson/UberJobMsg.h b/src/protojson/UberJobMsg.h index c4969bcfee..87c42cdba3 100644 --- a/src/protojson/UberJobMsg.h +++ b/src/protojson/UberJobMsg.h @@ -218,14 +218,14 @@ class JobMsg { JobId _jobId; int _attemptCount; - std::string _chunkQuerySpecDb; // &&& remove, use value for UJ + std::string _chunkQuerySpecDb; int _chunkId; JobFragment::VectPtr _jobFragments{new JobFragment::Vect()}; JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< Map of all query templates related to this UberJob. JobDbTablesMap::Ptr _jobDbTablesMap; ///< Map of all db.tables related to this UberJob. - // &&& remove, use value for UJ + // TODO:UJ remove, use value for UJ std::vector _chunkScanTableIndexes; ///< list of indexes into _jobDbTablesMap. }; diff --git a/src/protojson/WorkerCzarComIssue.cc b/src/protojson/WorkerCzarComIssue.cc new file mode 100644 index 0000000000..1a9a10bbd9 --- /dev/null +++ b/src/protojson/WorkerCzarComIssue.cc @@ -0,0 +1,117 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "protojson/WorkerCzarComIssue.h" + +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "util/common.h" +#include "util/TimeUtils.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.WorkerCzarComIssue"); +} // namespace + +namespace lsst::qserv::protojson { + +shared_ptr WorkerCzarComIssue::serializeJson() { + shared_ptr jsCzarReqPtr = make_shared(); + json& jsCzarR = *jsCzarReqPtr; + lock_guard _lgWciMtx(_wciMtx); + if (_wInfo == nullptr || _czInfo == nullptr) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _wInfo or _czInfo was null"); + return jsCzarReqPtr; + } + + jsCzarR["version"] = http::MetaModule::version; + jsCzarR["instance_id"] = _replicationInstanceId; + jsCzarR["auth_key"] = _replicationAuthKey; + jsCzarR["czarinfo"] = _czInfo->serializeJson(); + jsCzarR["czar"] = _czInfo->czName; + jsCzarR["workerinfo"] = _wInfo->serializeJson(); + + jsCzarR["thoughtczarwasdead"] = _thoughtCzarWasDead; + + // TODO:UJ add list of failed transmits + + return jsCzarReqPtr; +} + +WorkerCzarComIssue::Ptr WorkerCzarComIssue::createFromJson(nlohmann::json const& jsCzarReq, + std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_) { + string const fName("WorkerCzarComIssue::createFromJson"); + LOGS(_log, LOG_LVL_DEBUG, fName); + try { + if (jsCzarReq["version"] != http::MetaModule::version) { + LOGS(_log, LOG_LVL_ERROR, fName << " bad version"); + return nullptr; + } + + auto czInfo_ = CzarContactInfo::createFromJson(jsCzarReq["czarinfo"]); + auto now = CLOCK::now(); + auto wInfo_ = WorkerContactInfo::createFromJsonWorker(jsCzarReq["workerinfo"], now); + if (czInfo_ == nullptr || wInfo_ == nullptr) { + LOGS(_log, LOG_LVL_ERROR, fName << " or worker info could not be parsed in " << jsCzarReq); + } + auto wccIssue = create(replicationInstanceId_, replicationAuthKey_); + wccIssue->setContactInfo(wInfo_, czInfo_); + wccIssue->_thoughtCzarWasDead = + http::RequestBodyJSON::required(jsCzarReq, "thoughtczarwasdead"); + return wccIssue; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); + } + return nullptr; +} + +json WorkerCzarComIssue::serializeResponseJson() { + json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; + + // TODO:UJ add lists of uberjobs that are scheduled to have files collected because of this message. + return jsResp; +} + +string WorkerCzarComIssue::dump() const { + lock_guard _lgWciMtx(_wciMtx); + return _dump(); +} + +string WorkerCzarComIssue::_dump() const { + stringstream os; + os << "WorkerCzarComIssue wInfo=" << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); + os << " czInfo=" << _czInfo->dump(); + os << " thoughtCzarWasDead=" << _thoughtCzarWasDead; + return os.str(); +} + +} // namespace lsst::qserv::protojson diff --git a/src/protojson/WorkerCzarComIssue.h b/src/protojson/WorkerCzarComIssue.h new file mode 100644 index 0000000000..21a0e53cdb --- /dev/null +++ b/src/protojson/WorkerCzarComIssue.h @@ -0,0 +1,131 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_PROTOJSON_WORKERCZARCOMISSUE_H +#define LSST_QSERV_PROTOJSON_WORKERCZARCOMISSUE_H + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "protojson/WorkerQueryStatusData.h" + +// This header declarations +namespace lsst::qserv::protojson { + +/// This class is used to send/receive a message from the worker to a specific +/// czar when there has been a communication issue with the worker sending UberJob +/// file ready messages. If there have been timeouts, the worker will send this +/// message to the czar immediately after the worker receives a +/// WorkerQueryStatusData message from the czar (indicating that communication +/// is now possible). +/// If communication with the czar has failed for a long time, the worker +/// will set "_thoughtCzarWasDead" and delete all incomplete work associated +/// with that czar. Result files will remain until garbage cleanup or the czar +/// calls for their removal. +/// TODO:UJ UberJob complete messages that failed to be sent to the czar +/// TODO:UJ will be added to this message. (uberjob file response) +/// Upon successful completion, the worker will clear all values set by the +/// the czar. +/// Currently, this message is expected to only be needed rarely. +class WorkerCzarComIssue { +public: + using Ptr = std::shared_ptr; + + WorkerCzarComIssue() = delete; + ~WorkerCzarComIssue() = default; + + std::string cName(const char* funcN) { return std::string("WorkerCzarComIssue") + funcN; } + + static Ptr create(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { + return Ptr(new WorkerCzarComIssue(replicationInstanceId_, replicationAuthKey_)); + } + + static Ptr createFromJson(nlohmann::json const& workerJson, std::string const& replicationInstanceId_, + std::string const& replicationAuthKey_); + + void setThoughtCzarWasDead(bool wasDead) { + std::lock_guard lg(_wciMtx); + _thoughtCzarWasDead = wasDead; + } + + bool getThoughtCzarWasDead() const { return _thoughtCzarWasDead; } + + /// Return true if there is a reason this WorkerCzarComIssue should be sent to this czar. + bool needToSend() const { + std::lock_guard lg(_wciMtx); + // TODO:UJ or list of failed transmits not empty. + return _thoughtCzarWasDead; + } + + /// Set the contact information for the appropriate czar and worker. + void setContactInfo(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_) { + std::lock_guard lgWci(_wciMtx); + if (_wInfo == nullptr && wInfo_ != nullptr) _wInfo = wInfo_; + if (_czInfo == nullptr && czInfo_ != nullptr) _czInfo = czInfo_; + } + + CzarContactInfo::Ptr getCzarInfo() const { + std::lock_guard lgWci(_wciMtx); + return _czInfo; + } + + WorkerContactInfo::Ptr getWorkerInfo() const { + std::lock_guard lgWci(_wciMtx); + return _wInfo; + } + + /// Return a json version of the contents of this class. + std::shared_ptr serializeJson(); + + /// Return a json object indicating the status of the message for the + /// original requester. + nlohmann::json serializeResponseJson(); + + std::string dump() const; + +private: + WorkerCzarComIssue(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) + : _replicationInstanceId(replicationInstanceId_), _replicationAuthKey(replicationAuthKey_) {} + + std::string _dump() const; + + WorkerContactInfo::Ptr _wInfo; + CzarContactInfo::Ptr _czInfo; + std::string const _replicationInstanceId; ///< Used for message verification. + std::string const _replicationAuthKey; ///< Used for message verification. + + /// Set to by the worker true if the czar was considered dead, and reset to false + /// after the czar has acknowledged successful reception of this message. + bool _thoughtCzarWasDead = false; + + mutable MUTEX _wciMtx; ///< protects all members. +}; + +} // namespace lsst::qserv::protojson + +#endif // LSST_QSERV_PROTOJSON_WORKERCZARCOMISSUE_H diff --git a/src/protojson/WorkerQueryStatusData.cc b/src/protojson/WorkerQueryStatusData.cc index ea3916b6fa..ed4916812d 100644 --- a/src/protojson/WorkerQueryStatusData.cc +++ b/src/protojson/WorkerQueryStatusData.cc @@ -416,7 +416,7 @@ bool WorkerQueryStatusData::handleResponseJson(nlohmann::json const& jsResp) { bool workerRestarted = false; auto workerStartupTime = http::RequestBodyJSON::required(jsResp, "w-startup-time"); - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " workerStartupTime=" << workerStartupTime); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " workerStartupTime=" << workerStartupTime); if (!_wInfo->checkWStartupTime(workerStartupTime)) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " startup time for worker=" << _wInfo->dump() @@ -438,75 +438,4 @@ string WorkerQueryStatusData::_dump() const { return os.str(); } -shared_ptr WorkerCzarComIssue::serializeJson() { - shared_ptr jsCzarReqPtr = make_shared(); - json& jsCzarR = *jsCzarReqPtr; - lock_guard _lgWciMtx(_wciMtx); - if (_wInfo == nullptr || _czInfo == nullptr) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _wInfo or _czInfo was null"); - return jsCzarReqPtr; - } - - jsCzarR["version"] = http::MetaModule::version; - jsCzarR["instance_id"] = _replicationInstanceId; - jsCzarR["auth_key"] = _replicationAuthKey; - jsCzarR["czarinfo"] = _czInfo->serializeJson(); - jsCzarR["czar"] = _czInfo->czName; - jsCzarR["workerinfo"] = _wInfo->serializeJson(); - - jsCzarR["thoughtczarwasdead"] = _thoughtCzarWasDead; - - // TODO:UJ add list of failed transmits - - return jsCzarReqPtr; -} - -WorkerCzarComIssue::Ptr WorkerCzarComIssue::createFromJson(nlohmann::json const& jsCzarReq, - std::string const& replicationInstanceId_, - std::string const& replicationAuthKey_) { - string const fName("WorkerCzarComIssue::createFromJson"); - LOGS(_log, LOG_LVL_DEBUG, fName); - try { - if (jsCzarReq["version"] != http::MetaModule::version) { - LOGS(_log, LOG_LVL_ERROR, fName << " bad version"); - return nullptr; - } - - auto czInfo_ = CzarContactInfo::createFromJson(jsCzarReq["czarinfo"]); - auto now = CLOCK::now(); - auto wInfo_ = WorkerContactInfo::createFromJsonWorker(jsCzarReq["workerinfo"], now); - if (czInfo_ == nullptr || wInfo_ == nullptr) { - LOGS(_log, LOG_LVL_ERROR, fName << " or worker info could not be parsed in " << jsCzarReq); - } - auto wccIssue = create(replicationInstanceId_, replicationAuthKey_); - wccIssue->setContactInfo(wInfo_, czInfo_); - wccIssue->_thoughtCzarWasDead = - http::RequestBodyJSON::required(jsCzarReq, "thoughtczarwasdead"); - return wccIssue; - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("WorkerQueryStatusData::createJson invalid ") << exc.what()); - } - return nullptr; -} - -json WorkerCzarComIssue::serializeResponseJson() { - json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; - - // TODO:UJ add lists of uberjobs that are scheduled to have files collected because of this message. - return jsResp; -} - -string WorkerCzarComIssue::dump() const { - lock_guard _lgWciMtx(_wciMtx); - return _dump(); -} - -string WorkerCzarComIssue::_dump() const { - stringstream os; - os << "WorkerCzarComIssue wInfo=" << ((_wInfo == nullptr) ? "?" : _wInfo->dump()); - os << " czInfo=" << _czInfo->dump(); - os << " thoughtCzarWasDead=" << _thoughtCzarWasDead; - return os.str(); -} - } // namespace lsst::qserv::protojson diff --git a/src/protojson/WorkerQueryStatusData.h b/src/protojson/WorkerQueryStatusData.h index eeb93f2847..82599bf673 100644 --- a/src/protojson/WorkerQueryStatusData.h +++ b/src/protojson/WorkerQueryStatusData.h @@ -374,105 +374,6 @@ class WorkerQueryStatusData { std::string _dump() const; }; -/// This class is used to send/receive a message from the worker to a specific -/// czar when there has been a communication issue with the worker sending UberJob -/// file ready messages. If there have been timeouts, the worker will send this -/// message to the czar immediately after the worker receives a -/// WorkerQueryStatusData message from the czar (indicating that communication -/// is now possible). -/// If communication with the czar has failed for a long time, the worker -/// will set "_thoughtCzarWasDead" and delete all incomplete work associated -/// with that czar. Result files will remain until garbage cleanup or the czar -/// calls for their removal. -/// TODO:UJ &&& UberJob complete messages that failed to be sent to the czar -/// TODO:UJ &&& will be added to this message. (uberjob file response) -/// Upon successful completion, the worker will clear all values set by the -/// the czar. -/// Currently, this message is expected to only be needed rarely. -class WorkerCzarComIssue { -public: - using Ptr = std::shared_ptr; - - WorkerCzarComIssue() = delete; - ~WorkerCzarComIssue() = default; - - std::string cName(const char* funcN) { return std::string("WorkerCzarComIssue") + funcN; } - - static Ptr create(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { - return Ptr(new WorkerCzarComIssue(replicationInstanceId_, replicationAuthKey_)); - } - - static Ptr createFromJson(nlohmann::json const& workerJson, std::string const& replicationInstanceId_, - std::string const& replicationAuthKey_); - - void setThoughtCzarWasDead(bool wasDead) { - std::lock_guard lg(_wciMtx); - _thoughtCzarWasDead = wasDead; - } - - bool getThoughtCzarWasDead() const { return _thoughtCzarWasDead; } - - /// Return true if there is a reason this WorkerCzarComIssue should be sent to this czar. - bool needToSend() const { - std::lock_guard lg(_wciMtx); - // TODO:UJ &&& or list of failed transmits not empty. - return _thoughtCzarWasDead; - } - - /// Set the contact information for the appropriate czar and worker. - void setContactInfo(WorkerContactInfo::Ptr const& wInfo_, CzarContactInfo::Ptr const& czInfo_) { - std::lock_guard lgWci(_wciMtx); - if (_wInfo == nullptr && wInfo_ != nullptr) _wInfo = wInfo_; - if (_czInfo == nullptr && czInfo_ != nullptr) _czInfo = czInfo_; - } - - CzarContactInfo::Ptr getCzarInfo() const { - std::lock_guard lgWci(_wciMtx); - return _czInfo; - } - - WorkerContactInfo::Ptr getWorkerInfo() const { - std::lock_guard lgWci(_wciMtx); - return _wInfo; - } - - /// Return a json version of the contents of this class. - std::shared_ptr serializeJson(); - - /// Return a json object indicating the status of the message for the - /// original requester. - nlohmann::json serializeResponseJson(); - - std::string dump() const; - -private: - WorkerCzarComIssue(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) - : _replicationInstanceId(replicationInstanceId_), _replicationAuthKey(replicationAuthKey_) {} - - std::string _dump() const; - - WorkerContactInfo::Ptr _wInfo; - CzarContactInfo::Ptr _czInfo; - std::string const _replicationInstanceId; ///< Used for message verification. - std::string const _replicationAuthKey; ///< Used for message verification. - - /// Set to by the worker true if the czar was considered dead, and reset to false - /// after the czar has acknowledged successful reception of this message. - bool _thoughtCzarWasDead = false; - - mutable MUTEX _wciMtx; ///< protects all members. -}; - -class WorkerUberJobMsg { -public: - using Ptr = std::shared_ptr; - - static Ptr create(); - -private: - WorkerUberJobMsg(); -}; - } // namespace lsst::qserv::protojson #endif // LSST_QSERV_PROTOJSON_WORKERQUERYSTATUSDATA_H diff --git a/src/protojson/testJobReadyMsg.cc b/src/protojson/testJobReadyMsg.cc new file mode 100644 index 0000000000..339bd8e3d3 --- /dev/null +++ b/src/protojson/testJobReadyMsg.cc @@ -0,0 +1,106 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include +#include +#include + +#include "nlohmann/json.hpp" + +// Qserv headers +#include "global/clock_defs.h" +#include "http/MetaModule.h" +#include "lsst/log/Log.h" +#include "protojson/JobReadyMsg.h" + +// Boost unit test header +#define BOOST_TEST_MODULE RequestQuery +#include + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.testJobReadyMsg"); +} + +using namespace std; +namespace test = boost::test_tools; +using namespace lsst::qserv::protojson; + +string const repliInstanceId = "repliInstId"; +string const repliAuthKey = "repliIAuthKey"; + +BOOST_AUTO_TEST_SUITE(Suite) + +bool parseSerializeReparseCheck(string const& jsStr, string const& note) { + string fName("parseSerialize "); + fName += note + " "; + LOGS(_log, LOG_LVL_INFO, fName << " start " << jsStr); + nlohmann::json js = nlohmann::json::parse(jsStr); + LOGS(_log, LOG_LVL_INFO, fName << " parse 1"); + + JobReadyMsg::Ptr jrm = JobReadyMsg::createFromJson(js, repliInstanceId, repliAuthKey); + BOOST_REQUIRE(jrm != nullptr); + + nlohmann::json jsJrm = jrm->serializeJson(); + LOGS(_log, LOG_LVL_INFO, fName << " serialized jsJrm=" << jsJrm); + + JobReadyMsg::Ptr jrmCreated = JobReadyMsg::createFromJson(jsJrm, repliInstanceId, repliAuthKey); + LOGS(_log, LOG_LVL_INFO, fName << " created"); + nlohmann::json jsJrmCreated = jrmCreated->serializeJson(); + LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); + + bool createdMatchesOriginal = jsJrm == jsJrmCreated; + if (createdMatchesOriginal) { + LOGS(_log, LOG_LVL_INFO, fName << "created matches original"); + } else { + LOGS(_log, LOG_LVL_ERROR, "jsJrm != jsJrmCreated"); + LOGS(_log, LOG_LVL_ERROR, "jsJrm=" << jsJrm); + LOGS(_log, LOG_LVL_ERROR, "jsJrmCreated=" << jsJrmCreated); + } + BOOST_REQUIRE(createdMatchesOriginal); + return createdMatchesOriginal; +} + +BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { + LOGS(_log, LOG_LVL_INFO, "testJRM start"); + + string const workerIdStr("wrker72"); + string const czarName("cz4242"); + lsst::qserv::CzarIdType const czarId = 745; + lsst::qserv::QueryId const queryId = 986532; + lsst::qserv::UberJobId const uberJobId = 14578; + string const fileUrl("ht.qwrk/some/dir/fil.txt"); + uint64_t const rowCount = 391; + uint64_t const fileSize = 5623; + + auto jrm = JobReadyMsg::create(repliInstanceId, repliAuthKey, workerIdStr, czarName, czarId, queryId, + uberJobId, fileUrl, rowCount, fileSize); + + auto jsJrm = jrm->serializeJson(); + string const strJrm = to_string(jsJrm); + LOGS(_log, LOG_LVL_INFO, "stdJrm=" << strJrm); + + BOOST_REQUIRE(parseSerializeReparseCheck(strJrm, "A")); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/protojson/testStatusData.cc b/src/protojson/testStatusData.cc index 8dd226080d..9b63827ea7 100644 --- a/src/protojson/testStatusData.cc +++ b/src/protojson/testStatusData.cc @@ -29,6 +29,7 @@ // Qserv headers #include "global/clock_defs.h" #include "lsst/log/Log.h" +#include "protojson/WorkerCzarComIssue.h" #include "protojson/WorkerQueryStatusData.h" // Boost unit test header diff --git a/src/proxy/CMakeLists.txt b/src/proxy/CMakeLists.txt index 5aa1ca3e2c..03dd2fe1f2 100644 --- a/src/proxy/CMakeLists.txt +++ b/src/proxy/CMakeLists.txt @@ -27,7 +27,7 @@ target_link_libraries(czarProxy PRIVATE rproc css qmeta - xrdreq) +) install(TARGETS czarProxy DESTINATION lua/qserv/lib) install(FILES mysqlProxy.lua DESTINATION lua/qserv/scripts) diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index f15024d7a2..e19de24081 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -10,15 +10,10 @@ target_sources(qdisp PRIVATE UberJob.cc ) -target_include_directories(qdisp PRIVATE - ${XROOTD_INCLUDE_DIRS} -) target_link_libraries(qdisp PUBLIC cconfig log - http - XrdSsiLib ) install( @@ -27,14 +22,11 @@ install( add_executable(testQDisp testQDisp.cc) -target_include_directories(testQDisp PRIVATE - ${XROOTD_INCLUDE_DIRS} -) - target_link_libraries(testQDisp cconfig ccontrol czar + mysql parser qana qdisp diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 21b22f17b8..9b47721961 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -261,14 +261,6 @@ void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { } } -void Executive::queueFileCollect(util::PriorityCommand::Ptr const& cmd) { - if (_scanInteractive) { - _qdispPool->queCmd(cmd, 3); - } else { - _qdispPool->queCmd(cmd, 3); - } -} - void Executive::addAndQueueUberJob(shared_ptr const& uj) { { lock_guard lck(_uberJobsMapMtx); @@ -349,63 +341,6 @@ void Executive::addMultiError(int errorCode, std::string const& errorMsg, int er } } -Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { - lock_guard lck(_chunkToJobMapMtx); - - ChunkIdJobMapType unassignedMap; - for (auto const& [key, jobPtr] : _chunkToJobMap) { - if (!jobPtr->isInUberJob()) { - unassignedMap[key] = jobPtr; - } - } - return unassignedMap; -} - -void Executive::addUberJobs(std::vector> const& uJobsToAdd) { - lock_guard lck(_uberJobsMapMtx); - for (auto const& uJob : uJobsToAdd) { - UberJobId ujId = uJob->getJobId(); - _uberJobsMap[ujId] = uJob; - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " ujId=" << ujId << " uj.sz=" << uJob->getJobCount()); - } -} - -string Executive::dumpUberJobCounts() const { - stringstream os; - os << "exec=" << getIdStr(); - int totalJobs = 0; - { - lock_guard ujmLck(_uberJobsMapMtx); - for (auto const& [ujKey, ujPtr] : _uberJobsMap) { - int jobCount = ujPtr->getJobCount(); - totalJobs += jobCount; - os << "{" << ujKey << ":" << ujPtr->getIdStr() << " jobCount=" << jobCount << "}"; - } - } - { - lock_guard jmLck(_jobMapMtx); - os << " ujTotalJobs=" << totalJobs << " execJobs=" << _jobMap.size(); - } - return os.str(); -} - -void Executive::assignJobsToUberJobs() { - auto uqs = _userQuerySelect.lock(); - if (uqs != nullptr) { - uqs->buildAndSendUberJobs(); - } -} - -void Executive::addMultiError(int errorCode, std::string const& errorMsg, int errorState) { - util::Error err(errorCode, errorMsg, errorState); - { - lock_guard lock(_errorsMutex); - _multiError.push_back(err); - LOGS(_log, LOG_LVL_DEBUG, - cName(__func__) + " multiError:" << _multiError.size() << ":" << _multiError); - } -} - /// Add a JobQuery to this Executive. /// Return true if it was successfully added to the map. /// @@ -458,17 +393,14 @@ bool Executive::join() { } void Executive::markCompleted(JobId jobId, bool success) { - ResponseHandler::Error err; + string errStr; + util::Error err; string idStr = QueryIdHelper::makeIdStr(_id, jobId); LOGS(_log, LOG_LVL_DEBUG, "Executive::markCompleted " << success); if (!success && !isRowLimitComplete()) { { lock_guard lock(_incompleteJobsMutex); - auto iter = _incompleteJobs.find(jobId); - if (iter != _incompleteJobs.end()) { - auto jobQuery = iter->second; - err = jobQuery->getDescription()->respHandler()->getError(); - } else { + if (_incompleteJobs.count(jobId) == 0) { string msg = "Executive::markCompleted failed to find TRACKED " + idStr + " size=" + to_string(_incompleteJobs.size()); // If the user query has been cancelled, this is expected for jobs that have not yet @@ -482,8 +414,14 @@ void Executive::markCompleted(JobId jobId, bool success) { return; } } - LOGS(_log, LOG_LVL_WARN, - "Executive: error executing " << err << " (status: " << err.getStatus() << ")"); + + { + lock_guard lock(_errorsMutex); + errStr = _multiError.firstErrorStr(); + err = _multiError.firstError(); + } + + LOGS(_log, LOG_LVL_DEBUG, "Executive: error executing " << err); { lock_guard lockJobMap(_jobMapMtx); auto job = _jobMap[jobId]; @@ -493,20 +431,10 @@ void Executive::markCompleted(JobId jobId, bool success) { job->getStatus()->updateInfoNoErrorOverwrite(id, qmeta::JobStatus::RESULT_ERROR, "EXECFAIL", err.getCode(), err.getMsg()); } - { - lock_guard lock(_errorsMutex); - _multiError.push_back(err); - LOGS(_log, LOG_LVL_TRACE, - "Currently " << _multiError.size() << " registered errors: " << _multiError); - } } _unTrack(jobId); if (!success && !isRowLimitComplete()) { - auto logLvl = (_cancelled) ? LOG_LVL_ERROR : LOG_LVL_TRACE; - LOGS(_log, logLvl, - "Executive: requesting cancel, cause: " << " failed (code=" << err.getCode() << " " - << err.getMsg() << ")"); - squash(string("markComplete error ") + err.getMsg()); // ask to squash + squash("markComplete error " + errStr); // ask to squash } } @@ -618,44 +546,6 @@ void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { } } -void Executive::sendWorkersEndMsg(bool deleteResults) { - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " terminating this query deleteResults=" << deleteResults); - auto cz = czar::Czar::getCzar(); - if (cz != nullptr) { // Possible in unit tests. - cz->getCzarRegistry()->endUserQueryOnWorkers(_id, deleteResults); - } -} - -void Executive::killIncompleteUberJobsOnWorker(std::string const& workerId) { - if (_cancelled) { - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " irrelevant as query already cancelled"); - return; - } - - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " killing incomplete UberJobs on " << workerId); - deque ujToCancel; - { - lock_guard lockUJMap(_uberJobsMapMtx); - for (auto const& [ujKey, ujPtr] : _uberJobsMap) { - auto ujStatus = ujPtr->getStatus()->getState(); - if (ujStatus != qmeta::JobStatus::RESPONSE_DONE && ujStatus != qmeta::JobStatus::COMPLETE) { - // RESPONSE_DONE indicates the result file has been read by - // the czar, so before that point the worker's data is - // likely destroyed. COMPLETE indicates all jobs in the - // UberJob are complete. - if (ujPtr->getWorkerContactInfo()->wId == workerId) { - ujToCancel.push_back(ujPtr); - } - } - } - } - - for (auto const& uj : ujToCancel) { - uj->killUberJob(); - uj->setStatusIfOk(qmeta::JobStatus::CANCEL, getIdStr() + " killIncomplete on worker=" + workerId); - } -} - int Executive::getNumInflight() const { unique_lock lock(_incompleteJobsMutex); return _incompleteJobs.size(); @@ -909,8 +799,8 @@ void Executive::checkResultFileSize(uint64_t fileSize) { cName(__func__) << "recheck total=" << total << " max=" << maxResultTableSizeBytes); if (total > maxResultTableSizeBytes) { LOGS(_log, LOG_LVL_ERROR, "Executive: requesting squash, result file size too large " << total); - ResponseHandler::Error err(util::ErrorCode::CZAR_RESULT_TOO_LARGE, - string("Incomplete result already too large ") + to_string(total)); + util::Error err(util::ErrorCode::CZAR_RESULT_TOO_LARGE, + "Incomplete result already too large " + to_string(total)); _multiError.push_back(err); squash("czar, file too large"); } diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 583707c7a6..346cd7f6a5 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -85,7 +85,6 @@ class QdispPool; // This header declarations namespace lsst::qserv::qdisp { - /// class Executive manages the execution of jobs for a UserQuery. class Executive : public std::enable_shared_from_this { public: @@ -118,20 +117,6 @@ class Executive : public std::enable_shared_from_this { /// Find the UberJob with `ujId`. std::shared_ptr findUberJob(UberJobId ujId); - std::string cName(const char* funcName = "") { - return std::string("Executive::") + funcName + " " + getIdStr(); - } - - /// Set the UserQuerySelect object for this query so this Executive can ask it to make new - /// UberJobs in the future, if needed. - void setUserQuerySelect(std::shared_ptr const& uqs) { _userQuerySelect = uqs; } - - /// Return a map that only contains Jobs not assigned to an UberJob. - ChunkIdJobMapType unassignedChunksInQuery(); - - /// Find the UberJob with `ujId`. - std::shared_ptr findUberJob(UberJobId ujId); - /// Add an item with a reference number std::shared_ptr add(JobDescription::Ptr const& s); @@ -141,9 +126,6 @@ class Executive : public std::enable_shared_from_this { /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. void queueFileCollect(std::shared_ptr const& cmd); - /// Queue `cmd`, using the QDispPool, so it can be used to collect the result file. - void queueFileCollect(std::shared_ptr const& cmd); - /// Waits for all jobs on _jobStartCmdList to start. This should not be called /// before ALL jobs have been added to the pool. void waitForAllJobsToStart(); @@ -218,9 +200,6 @@ class Executive : public std::enable_shared_from_this { /// Add an error code and message that may be displayed to the user. void addMultiError(int errorCode, std::string const& errorMsg, int errState); - /// Add an error code and message that may be displayed to the user. - void addMultiError(int errorCode, std::string const& errorMsg, int errState); - std::string dumpUberJobCounts() const; // The below value should probably be based on the user query, with longer sleeps for slower queries. @@ -285,6 +264,7 @@ class Executive : public std::enable_shared_from_this { /// The stats are pushed to qdisp::CzarStats. void _updateStats() const; + util::InstanceCount const _icEx{"Executive"}; std::atomic _empty{true}; std::shared_ptr _messageStore; ///< MessageStore for logging @@ -346,18 +326,6 @@ class Executive : public std::enable_shared_from_this { std::map> _uberJobsMap; mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. - // Add a job to the _chunkToJobMap - // TODO:UJ This may need review as large changes were made to this part of the code. - // code is no longer destructive to _chunkToJobMap - void _addToChunkJobMap(std::shared_ptr const& job); - std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap - ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId - - /// Map of all UberJobs. Failed UberJobs remain in the map as new ones are created - /// to handle failed UberJobs. - std::map> _uberJobsMap; - mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. - /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. std::atomic _rowLimitComplete{false}; diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index a5bd734157..eea45f440b 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -48,17 +48,13 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.JobDescription"); namespace lsst::qserv::qdisp { JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, - shared_ptr const& respHandler, - shared_ptr const& chunkQuerySpec, - string const& chunkResultName, bool mock) + shared_ptr const& chunkQuerySpec, bool mock) : _czarId(czarId), _queryId(qId), _jobId(jobId), _qIdStr(QueryIdHelper::makeIdStr(_queryId, _jobId)), _resource(resource), - _respHandler(respHandler), _chunkQuerySpec(chunkQuerySpec), - _chunkResultName(chunkResultName), _mock(mock) {} bool JobDescription::incrAttemptCount(std::shared_ptr const& exec, bool increase) { @@ -69,7 +65,7 @@ bool JobDescription::incrAttemptCount(std::shared_ptr const& exec, bo if (exec != nullptr) { int maxAttempts = exec->getMaxAttempts(); if (_attemptCount > 0) { - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " attempts=" << _attemptCount); + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " attempts=" << _attemptCount); } if (_attemptCount > maxAttempts) { LOGS(_log, LOG_LVL_ERROR, diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 72b0a059fa..df5d810f6a 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -60,25 +60,23 @@ class JobDescription { using Ptr = std::shared_ptr; static JobDescription::Ptr create(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, - std::shared_ptr const& respHandler, std::shared_ptr const& chunkQuerySpec, - std::string const& chunkResultName, bool mock = false) { - JobDescription::Ptr jd(new JobDescription(czarId, qId, jobId, resource, respHandler, chunkQuerySpec, - chunkResultName, mock)); + bool mock = false) { + JobDescription::Ptr jd(new JobDescription(czarId, qId, jobId, resource, chunkQuerySpec, mock)); return jd; } JobDescription(JobDescription const&) = delete; JobDescription& operator=(JobDescription const&) = delete; + virtual ~JobDescription() = default; + std::string cName(const char* fnc) { return std::string("JobDescription::") + fnc + " " + _qIdStr; } JobId id() const { return _jobId; } ResourceUnit const& resource() const { return _resource; } - std::shared_ptr respHandler() { return _respHandler; } int getAttemptCount() const { return _attemptCount; } std::shared_ptr getChunkQuerySpec() { return _chunkQuerySpec; } - std::string getChunkResultName() { return _chunkResultName; } bool getScanInteractive() const; int getScanRating() const; @@ -90,17 +88,11 @@ class JobDescription { void resetJsForWorker() { _jsForWorker.reset(); } - std::shared_ptr getJsForWorker() { return _jsForWorker; } - - void resetJsForWorker() { _jsForWorker.reset(); } - friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); private: JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, ResourceUnit const& resource, - std::shared_ptr const& respHandler, - std::shared_ptr const& chunkQuerySpec, - std::string const& chunkResultName, bool mock = false); + std::shared_ptr const& chunkQuerySpec, bool mock = false); qmeta::CzarId _czarId; QueryId _queryId; @@ -109,10 +101,7 @@ class JobDescription { int _attemptCount{-1}; ///< Start at -1 so that first attempt will be 0, see incrAttemptCount(). ResourceUnit _resource; ///< path, e.g. /q/LSST/23125 - // TODO:UJ delete _respHandler, store errors a different way - std::shared_ptr _respHandler; // probably MergingHandler std::shared_ptr _chunkQuerySpec; - std::string _chunkResultName; bool _mock{false}; ///< True if this is a mock in a unit test. diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index bf4246be49..9b6faa6b32 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -63,21 +63,17 @@ bool JobQuery::cancel(bool superfluous) { VMUTEX_NOT_HELD(_jqMtx); lock_guard lock(_jqMtx); - ostringstream os; - os << _idStr << " cancel"; - LOGS(_log, LOG_LVL_DEBUG, os.str()); - if (!superfluous) { - getDescription()->respHandler()->errorFlush(os.str(), -1); - } - auto executive = _executive.lock(); - if (executive == nullptr) { + string const context = _idStr + " cancel"; + LOGS(_log, LOG_LVL_DEBUG, context); + auto exec = _executive.lock(); + if (exec == nullptr) { LOGS(_log, LOG_LVL_ERROR, " can't markComplete cancelled, executive == nullptr"); return false; } - executive->markCompleted(getJobId(), false); if (!superfluous) { - _jobDescription->respHandler()->processCancel(); + exec->addMultiError(-1, context, util::ErrorCode::RESULT_IMPORT); } + exec->markCompleted(getJobId(), false); return true; } LOGS(_log, LOG_LVL_TRACE, "JobQuery::cancel, skipping, already cancelled."); @@ -110,10 +106,6 @@ bool JobQuery::_setUberJobId(UberJobId ujId) { return true; } -ostream& JobQuery::dumpOS(ostream& os) const { - return os << "{" << getIdStr() << _jobDescription << " " << _jobStatus << "}"; -} - bool JobQuery::unassignFromUberJob(UberJobId ujId) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); VMUTEX_NOT_HELD(_jqMtx); diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 607c76536d..b753c92853 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -119,23 +119,6 @@ class JobQuery { return _uberJobId >= 0; } - /// @return true if _uberJobId was set, it can only be set if it is unassigned - /// or by the current owner. - /// NOTE: _jqMtx must be held before calling this - bool _setUberJobId(UberJobId ujId); - - /// NOTE: _jqMtx must be held before calling this - UberJobId _getUberJobId() const { - VMUTEX_HELD(_jqMtx); - return _uberJobId; - } - - /// NOTE: _jqMtx must be held before calling this - bool _isInUberJob() const { - VMUTEX_HELD(_jqMtx); - return _uberJobId >= 0; - } - // Values that don't change once set. std::weak_ptr _executive; /// The job description needs to survive until the task is complete. diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 7c21b6175a..1dcf055e4f 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -53,7 +53,6 @@ class UberJob; /// segment of results. class ResponseHandler { public: - typedef util::Error Error; using BufPtr = std::shared_ptr>; typedef std::shared_ptr Ptr; @@ -77,32 +76,12 @@ class ResponseHandler { /// Add the error to the error output if it is the first error. virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; - /// Collect result data from the worker and merge it with the query result table. - /// @return success - true if the operation was successful - /// @return shouldCancel - if success was false, this being true indicates there - /// was an unrecoverable error in table writing and the query - /// should be cancelled. - virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, - uint64_t& resultRows) = 0; - - /// Add the error to the error output if it is the first error. - virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; - /// Signal an unrecoverable error condition. No further calls are expected. virtual void errorFlush(std::string const& msg, int code) = 0; /// Print a string representation of the receiver to an ostream virtual std::ostream& print(std::ostream& os) const = 0; - /// @return an error code and description - virtual Error getError() const = 0; - - /// Do anything that needs to be done if this job gets cancelled. - virtual void processCancel() {}; - - /// Scrub the results from jobId-attempt from the result table. - virtual void prepScrubResults(int jobId, int attempt) = 0; - std::weak_ptr getUberJob() { return _uberJob; } private: diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 299b697ef7..4e7822ec37 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -205,28 +205,6 @@ void UberJob::_unassignJobs() { _jobs.clear(); } -void UberJob::_unassignJobs() { - lock_guard lck(_jobsMtx); - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " exec is null"); - return; - } - for (auto&& job : _jobs) { - string jid = job->getIdStr(); - if (!job->unassignFromUberJob(getUjId())) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " could not unassign job=" << jid << " cancelling"); - exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "unable to re-assign " + jid, - util::ErrorCode::INTERNAL); - exec->squash(); - return; - } - LOGS(_log, LOG_LVL_DEBUG, - cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); - } - _jobs.clear(); -} - bool UberJob::isQueryCancelled() { auto exec = _executive.lock(); if (exec == nullptr) { @@ -263,24 +241,6 @@ bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg return true; } -void UberJob::callMarkCompleteFunc(bool success) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " success=" << success); - - // Overwriting errors is probably not a good idea. - if (currentState >= qmeta::JobStatus::CANCEL && currentState < qmeta::JobStatus::COMPLETE) { - LOGS(_log, LOG_LVL_WARN, - getIdStr() << "UberJob::" << __func__ << " already error current=" - << _jobStatus->stateStr(currentState) << " new=" << _jobStatus->stateStr(newState)); - return false; - } - - _jobStatus->updateInfo(getIdStr(), newState, msg); - for (auto&& jq : _jobs) { - jq->getStatus()->updateInfo(jq->getIdStr(), newState, msg); - } - return true; -} - void UberJob::callMarkCompleteFunc(bool success) { LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); @@ -556,4 +516,3 @@ std::string UberJob::dump() const { std::ostream& operator<<(std::ostream& os, UberJob const& uj) { return uj.dumpOS(os); } } // namespace lsst::qserv::qdisp - diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 254ee9d7e0..b6bc616473 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -92,9 +92,6 @@ class TestInfo : public ResponseHandler { } void flushHttpError(int errorCode, std::string const& errorMsg, int status) override {} void errorFlush(std::string const& msg, int code) override {}; - Error getError() const override { return util::Error(); } - void processCancel() override {}; - void prepScrubResults(int jobId, int attempt) override {}; /// Print a string representation of the receiver to an ostream std::ostream& print(std::ostream& os) const override { @@ -181,133 +178,13 @@ class ExecutiveUT : public Executive { } // namespace lsst::qserv::qdisp -namespace lsst::qserv::qdisp { - -class ExecutiveUT; - -class TestInfo : public ResponseHandler { -public: - using Ptr = std::shared_ptr; - - TestInfo() {} - virtual ~TestInfo() {} - - bool goWait() { - unique_lock ulock(_infoMtx); - _infoCV.wait(ulock, [this]() { return _go == true; }); - return _ok; - } - - void setGo(bool val) { - lock_guard lg(_infoMtx); - _go = val; - _infoCV.notify_all(); - } - - // virtual function that won't be needed - std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, - uint64_t& resultRows) override { - return {true, false}; - } - void flushHttpError(int errorCode, std::string const& errorMsg, int status) override {} - void errorFlush(std::string const& msg, int code) override {}; - Error getError() const override { return util::Error(); } - void processCancel() override {}; - void prepScrubResults(int jobId, int attempt) override {}; - - /// Print a string representation of the receiver to an ostream - std::ostream& print(std::ostream& os) const override { - os << "TestInfo ujCount=" << ujCount; - return os; - } - - atomic ujCount = 0; - -private: - bool _ok = true; - bool _go = true; - mutex _infoMtx; - condition_variable _infoCV; -}; - -/// Version of UberJob specifically for this unit test. -class UberJobUT : public UberJob { -public: - using PtrUT = std::shared_ptr; - - UberJobUT(std::shared_ptr const& executive, - std::shared_ptr const& respHandler, int queryId, int uberJobId, - qmeta::CzarId czarId, int rowLimit, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData, - TestInfo::Ptr const& testInfo_) - : UberJob(executive, respHandler, queryId, uberJobId, czarId, rowLimit, workerData), - testInfo(testInfo_) {} - - void runUberJob() override { - LOGS(_log, LOG_LVL_INFO, "runUberJob() chunkId=" << chunkId); - bool ok = testInfo->goWait(); - int c = -1; - if (ok) { - c = ++(testInfo->ujCount); - } - callMarkCompleteFunc(ok); - LOGS(_log, LOG_LVL_INFO, "runUberJob() end chunkId=" << chunkId << " c=" << c); - } - - TestInfo::Ptr testInfo; - int chunkId = -1; -}; - -/// Version of Executive specifically for this unit test. -class ExecutiveUT : public Executive { -public: - using PtrUT = shared_ptr; - - ~ExecutiveUT() override = default; - - ExecutiveUT(ExecutiveConfig const& cfg, shared_ptr const& ms, - util::QdispPool::Ptr const& qdispPool, shared_ptr const& qStatus, - shared_ptr const& querySession, TestInfo::Ptr const& testInfo_) - : Executive(cfg, ms, qdispPool, qStatus, querySession), testInfo(testInfo_) {} - - void assignJobsToUberJobs() override { - vector ujVect; - - // Make an UberJobUnitTest for each job - qdisp::Executive::ChunkIdJobMapType unassignedChunks = unassignedChunksInQuery(); - for (auto const& [chunkId, jqPtr] : unassignedChunks) { - auto exec = shared_from_this(); - PtrUT execUT = dynamic_pointer_cast(exec); - auto uJob = UberJobUT::PtrUT(new UberJobUT(execUT, testInfo, getId(), ujId++, czarId, rowLimit, - targetWorker, testInfo)); - uJob->chunkId = chunkId; - uJob->addJob(jqPtr); - ujVect.push_back(uJob); - } - - for (auto const& ujPtr : ujVect) { - addAndQueueUberJob(ujPtr); - } - LOGS(_log, LOG_LVL_INFO, "assignJobsToUberJobs() end"); - } - - CzarIdType czarId = 1; - UberJobId ujId = 1; - int rowLimit = 0; - czar::CzarChunkMap::WorkerChunksData::Ptr targetWorker = nullptr; - - TestInfo::Ptr testInfo; -}; - -} // namespace lsst::qserv::qdisp - qdisp::JobDescription::Ptr makeMockJobDescription(qdisp::Executive::Ptr const& ex, int sequence, ResourceUnit const& ru, std::string msg, std::shared_ptr const& mHandler) { auto cqs = std::make_shared(); // dummy, unused in this case. std::string chunkResultName = "dummyResultTableName"; qmeta::CzarId const czarId = 1; - auto job = qdisp::JobDescription::create(czarId, ex->getId(), sequence, ru, mHandler, cqs, - chunkResultName, true); + auto job = qdisp::JobDescription::create(czarId, ex->getId(), sequence, ru, cqs, true); return job; } @@ -338,10 +215,9 @@ std::shared_ptr executiveTest(qdisp::ExecutiveUT::PtrUT const& // Test class Executive::add // Modeled after ccontrol::UserQuery::submit() ResourceUnit ru; - std::string chunkResultName = "mock"; std::shared_ptr infileMerger; - ccontrol::MergingHandler::Ptr mh = - std::make_shared(infileMerger, chunkResultName); + ccontrol::MergingHandler::Ptr mh = std::make_shared(infileMerger, ex); + RequesterVector rv; for (int j = 0; j < copies; ++j) { rv.push_back(mh); diff --git a/src/qhttp/CMakeLists.txt b/src/qhttp/CMakeLists.txt index 48fd320b68..3fb3458e48 100644 --- a/src/qhttp/CMakeLists.txt +++ b/src/qhttp/CMakeLists.txt @@ -29,3 +29,5 @@ target_link_libraries(testqhttp PUBLIC ) add_test(NAME testqhttp COMMAND testqhttp -- --data=${CMAKE_CURRENT_SOURCE_DIR}/testdata/ --retries=2 --retry-delay=1 --threads=2 --client-threads=4) + +# set_tests_properties(testqhttp PROPERTIES WILL_FAIL 1) diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index 969971f5ed..987a28594f 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -1,6 +1,6 @@ add_library(qmeta SHARED) -target_sources(qserv_meta PRIVATE +target_sources(qmeta PRIVATE JobStatus.cc MessageStore.cc QMeta.cc @@ -16,12 +16,10 @@ target_sources(qserv_meta PRIVATE target_link_libraries(qmeta PUBLIC cconfig global + http mysql - qdisp log - util - mysqlclient_r - http + util ) install(TARGETS qmeta) diff --git a/src/qmeta/QMeta.h b/src/qmeta/QMeta.h index 8bd55c7a41..05d54526bb 100644 --- a/src/qmeta/QMeta.h +++ b/src/qmeta/QMeta.h @@ -328,22 +328,6 @@ class QMeta { std::chrono::time_point const& prevUpdateTime = std::chrono::time_point()) = 0; - /** - * Fetch the chunk map which was updated after the specified time point. - * @param prevUpdateTime The cut off time for the chunk map age. Note that the default - * value of the parameter represents the start time of the UNIX Epoch. Leaving the default - * value forces an attempt to read the map from the database if the one would exist - * in there. - * @return Return the most current chunk disposition or the empty object if the persistent - * map is older than it was requested.The result could be evaluated by calling - * method empty() on the result object. - * @throws EmptyTableError if the corresponding metadata table doesn't have any record - * @throws SqlError for any other error related to MySQL - */ - virtual QMetaChunkMap getChunkMap( - std::chrono::time_point const& prevUpdateTime = - std::chrono::time_point()) = 0; - protected: // Default constructor QMeta() {} diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index 67a5cb6357..e89c702c63 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -814,14 +814,10 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c string const& table = row[2]; unsigned int chunk = lsst::qserv::stoui(row[3]); size_t const size = stoull(row[4]); -#if 1 //&&& chunkMap.workers[worker][database][table].push_back(QMetaChunkMap::ChunkInfo{chunk, size}); LOGS(_log, LOG_LVL_TRACE, "QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table << " chunk=" << chunk << " sz=" << size); -#else //&&& - chunkMap.workers[worker][database][table].push_back(ChunkMap::ChunkInfo{chunk, size}); -#endif //&&& } chunkMap.updateTime = updateTime; } catch (exception const& ex) { @@ -835,13 +831,9 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock sql::SqlErrorObject errObj; sql::SqlResults results; string const tableName = "chunkMapStatus"; -#if 1 //&&& string const query = "SELECT UNIX_TIMESTAMP(`update_time`) FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; -#else //&&& - string const query = "SELECT `update_time` FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; -#endif //&&& LOGS(_log, LOG_LVL_DEBUG, "Executing query: " << query); if (!_conn->runQuery(query, results, errObj)) { LOGS(_log, LOG_LVL_ERROR, "query failed: " << query); @@ -866,11 +858,7 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock } } -#if 1 //&&& void QMetaMysql::_addQueryMessage(QueryId queryId, qmeta::QueryMessage const& qMsg, int& cancelCount, -#else //&&& -void QMetaMysql::_addQueryMessage(QueryId queryId, qdisp::QueryMessage const& qMsg, int& cancelCount, -#endif //&&& int& completeCount, int& execFailCount, map& msgCountMap) { // Don't add duplicate messages. if (qMsg.msgSource == "DUPLICATE") return; diff --git a/src/qmeta/QMetaMysql.h b/src/qmeta/QMetaMysql.h index c7c03f9919..0afb534be2 100644 --- a/src/qmeta/QMetaMysql.h +++ b/src/qmeta/QMetaMysql.h @@ -252,10 +252,6 @@ class QMetaMysql : public QMeta { QMetaChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = std::chrono::time_point()) override; - /// @see QMeta::getChunkMap - QMetaChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = - std::chrono::time_point()) override; - protected: /// Check that all necessary tables exist void _checkDb(); diff --git a/src/qproc/CMakeLists.txt b/src/qproc/CMakeLists.txt index e87e0b3979..a8d7366273 100644 --- a/src/qproc/CMakeLists.txt +++ b/src/qproc/CMakeLists.txt @@ -53,3 +53,6 @@ qproc_tests( testQueryAnaIn testQueryAnaOrderBy ) + +# set_tests_properties(testQueryAnaGeneral PROPERTIES WILL_FAIL 1) + diff --git a/src/replica/CMakeLists.txt b/src/replica/CMakeLists.txt index 034bc76987..8be3c55044 100644 --- a/src/replica/CMakeLists.txt +++ b/src/replica/CMakeLists.txt @@ -24,8 +24,7 @@ target_link_libraries(replica PUBLIC replica_util replica_worker css - xrdreq - xrdsvc + global XrdCl XrdSsiLib http @@ -125,3 +124,6 @@ add_subdirectory(requests) add_subdirectory(services) add_subdirectory(util) add_subdirectory(worker) + +# set_tests_properties(testIngestRequestMgr PROPERTIES WILL_FAIL 1) + diff --git a/src/rproc/CMakeLists.txt b/src/rproc/CMakeLists.txt index 2cd4f0765a..817a87e90a 100644 --- a/src/rproc/CMakeLists.txt +++ b/src/rproc/CMakeLists.txt @@ -11,6 +11,7 @@ target_link_libraries(rproc PUBLIC log ) -install(TARGETS rproc - EXPORT qserv + +install( + TARGETS rproc ) diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index bb21066f70..6176e751df 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -60,6 +60,7 @@ #include "proto/ProtoImporter.h" #include "proto/worker.pb.h" #include "qdisp/CzarStats.h" +#include "qdisp/UberJob.h" #include "qproc/DatabaseModels.h" #include "query/ColumnRef.h" #include "query/SelectStmt.h" @@ -185,12 +186,6 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response return true; } - std::unique_ptr semaLock; - if (_dbEngine != MYISAM) { - // needed for parallel merging with INNODB and MEMORY - semaLock.reset(new util::SemaLock(*_semaMgrConn)); - } - TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, bool success) { if (!success) return; @@ -205,19 +200,11 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response util::Timer virtFileT; virtFileT.start(); // UberJobs only get one attempt - int resultJobId = makeJobIdAttempt(uberJob->getUjId(), 0); - ProtoRowBuffer::Ptr pRowBuffer = std::make_shared( - responseData, resultJobId, _jobIdColName, _jobIdSqlType, _jobIdMysqlType); + ProtoRowBuffer::Ptr pRowBuffer = std::make_shared(responseData); std::string const virtFile = _infileMgr.prepareSrc(pRowBuffer); std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); virtFileT.stop(); - // If the job attempt is invalid, exit without adding rows. - // It will wait here if rows need to be deleted. - if (_invalidJobAttemptMgr.incrConcurrentMergeCount(resultJobId)) { - return true; - } - size_t const resultSize = responseData.transmitsize(); size_t tResultSize; { @@ -249,18 +236,9 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response } auto start = CLOCK::now(); - switch (_dbEngine) { - case MYISAM: - ret = _applyMysqlMyIsam(infileStatement, resultSize); - break; - case INNODB: - [[fallthrough]]; - case MEMORY: - ret = _applyMysqlInnoDb(infileStatement, resultSize); - break; - default: - throw std::invalid_argument("InfileMerger::_dbEngine is unknown =" + engineToStr(_dbEngine)); - } + + // Always using MyIsam + ret = _applyMysqlMyIsam(infileStatement, resultSize); auto end = CLOCK::now(); auto mergeDur = std::chrono::duration_cast(end - start); LOGS(_log, LOG_LVL_TRACE, @@ -270,10 +248,7 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response if (not ret) { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); } - _invalidJobAttemptMgr.decrConcurrentMergeCount(); - LOGS(_log, LOG_LVL_TRACE, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); - return ret; } diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index 9cc8704b02..aa8085c219 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -43,6 +43,8 @@ #include "util/Error.h" #include "util/EventThread.h" +#include "util/InstanceCount.h" + // Forward declarations namespace lsst::qserv { namespace mysql { @@ -108,9 +110,6 @@ class InfileMerger { /// Merge the result data collected over Http. bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); - /// Merge the result data collected over Http. - bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); - /// Indicate the merge for the job is complete. void mergeCompleteFor(int jobId); @@ -175,6 +174,8 @@ class InfileMerger { bool _applySqlLocal(std::string const& sql, sql::SqlResults& results); bool _applySqlLocal(std::string const& sql, sql::SqlResults& results, sql::SqlErrorObject& errObj); bool _sqlConnect(sql::SqlErrorObject& errObj); + + util::InstanceCount const _icIm{"InfileMerger"}; std::string _getQueryIdStr(); void _setQueryIdStr(std::string const& qIdStr); void _fixupTargetName(); diff --git a/src/sql/CMakeLists.txt b/src/sql/CMakeLists.txt index 9475c63403..3b337892fd 100644 --- a/src/sql/CMakeLists.txt +++ b/src/sql/CMakeLists.txt @@ -13,6 +13,10 @@ target_sources(sql PRIVATE statement.cc ) +install( + TARGETS sql +) + target_link_libraries(sql PUBLIC log ) diff --git a/src/util/Command.h b/src/util/Command.h index b1620f5b65..6d28f87271 100644 --- a/src/util/Command.h +++ b/src/util/Command.h @@ -39,7 +39,7 @@ namespace lsst::qserv::util { class Tracker { public: Tracker() {} - virtual ~Tracker() {} + virtual ~Tracker() = default; enum class Status { INPROGRESS, COMPLETE }; using Ptr = std::shared_ptr; void setComplete(); @@ -88,7 +88,7 @@ class Command { class CommandTracked : public Command, public Tracker { public: using Ptr = std::shared_ptr; - CommandTracked() = default; + CommandTracked() : Command(), Tracker() {} explicit CommandTracked(std::function func) : Command(func) {} ~CommandTracked() override = default; diff --git a/src/util/Error.h b/src/util/Error.h index ad11db124f..8f303ee088 100644 --- a/src/util/Error.h +++ b/src/util/Error.h @@ -74,8 +74,12 @@ struct ErrorCode { */ class Error { public: - Error(int code = ErrorCode::NONE, std::string const& msg = "", int status = ErrorCode::NONE, - bool logLvLErr = true); + explicit Error(int code, std::string const& msg = "", int status = ErrorCode::NONE, + bool logLvLErr = true); + + Error() = default; + Error(Error const&) = default; + Error& operator=(Error const&) = default; /** Overload output operator for current class * @@ -101,9 +105,9 @@ class Error { bool isNone() { return (_code == util::ErrorCode::NONE); } private: - int _code; + int _code = ErrorCode::NONE; std::string _msg; - int _status; + int _status = ErrorCode::NONE; }; } // namespace lsst::qserv::util diff --git a/src/util/InstanceCount.cc b/src/util/InstanceCount.cc index 9960137609..69ebc4485b 100644 --- a/src/util/InstanceCount.cc +++ b/src/util/InstanceCount.cc @@ -4,10 +4,13 @@ #include "util/InstanceCount.h" // System Headers +#include // LSST headers #include "lsst/log/Log.h" +// qserv headers + using namespace std; namespace { // File-scope helpers @@ -18,8 +21,17 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.util.InstanceCount"); namespace lsst::qserv::util { -std::map InstanceCount::_instances; -std::recursive_mutex InstanceCount::_mx; +InstanceCount::InstanceCountData InstanceCount::_icData; + +InstanceCount::InstanceCountData::InstanceCountData() { + std::cout << "InstanceCountData " << " mx=" << (void*)(&_mx) << " _inst=" << (void*)(&_instances) + << " t=" << (void*)(this) << endl; +} + +InstanceCount::InstanceCountData::~InstanceCountData() { + cout << "~InstanceCountData " << " mx=" << (void*)(&_mx) << " _inst=" << (void*)(&_instances) + << " t=" << (void*)(this) << endl; +} InstanceCount::InstanceCount(std::string const& className) : _className{className} { _increment("con"); } @@ -28,46 +40,73 @@ InstanceCount::InstanceCount(InstanceCount const& other) : _className{other._cla InstanceCount::InstanceCount(InstanceCount&& origin) : _className{origin._className} { _increment("mov"); } void InstanceCount::_increment(std::string const& source) { - std::lock_guard lg(_mx); + std::lock_guard lg(_icData._mx); + static std::atomic first = true; + static InstanceCountData* icD = nullptr; + if (first.exchange(false) == true) { + icD = &_icData; + LOGS(_log, LOG_LVL_DEBUG, "InstanceCount::_increment first icd changed to " << (void*)icD); + } else { + if (icD != &_icData) { + LOGS(_log, LOG_LVL_ERROR, + "InstanceCount::_increment icd changed to " << (void*)&_icData << " from " << (void*)icD); + } + } std::pair entry(_className, 0); - auto ret = _instances.insert(entry); + auto ret = _icData._instances.insert(entry); auto iter = ret.first; iter->second += 1; LOGS(_log, LOG_LVL_TRACE, "InstanceCount " << source << " " << iter->first << "=" << iter->second); + if ((++(_icData._instanceLogLimiter)) % 10000 == 0) { + LOGS(_log, LOG_LVL_DEBUG, "InstanceCount brief " << *this << " icD=" << (void*)(&_icData)); + } } -uint16_t instanceDestructLogLimiter = 0; - InstanceCount::~InstanceCount() { - std::lock_guard lg(_mx); - ++instanceDestructLogLimiter; - auto iter = _instances.find(_className); - if (iter != _instances.end()) { + std::lock_guard lg(_icData._mx); + static std::atomic first = true; + static InstanceCountData* icD = nullptr; + if (first.exchange(false) == true) { + icD = &_icData; + LOGS(_log, LOG_LVL_DEBUG, "~InstanceCount first icd changed to " << (void*)icD); + } else { + if (icD != &_icData) { + LOGS(_log, LOG_LVL_ERROR, + "~InstanceCount icd changed to " << (void*)&_icData << " from " << (void*)icD); + } + } + auto iter = _icData._instances.find(_className); + if (iter != _icData._instances.end()) { iter->second -= 1; LOGS(_log, LOG_LVL_TRACE, "~InstanceCount " << iter->first << "=" << iter->second << " : " << *this); - if (instanceDestructLogLimiter % 1000 == 0) { - LOGS(_log, LOG_LVL_DEBUG, "~InstanceCount brief " << *this); + int sec = iter->second; + if (sec == 0 || (sec <= 100000 && sec % 1000 == 0) || (sec > 100000 && sec % 100000 == 0)) { + LOGS(_log, LOG_LVL_DEBUG, + "~InstanceCount " << iter->first << "=" << iter->second << " : " << *this + << " icD=" << (void*)(&_icData)); } - if (iter->second == 0) { - _instances.erase(_className); + if (sec == 0) { + _icData._instances.erase(iter); } } else { - LOGS(_log, LOG_LVL_ERROR, "~InstanceCount " << _className << " was not found! : " << *this); + LOGS(_log, LOG_LVL_ERROR, + "~InstanceCount " << _className << " was not found! : " << *this + << " icD=" << (void*)(&_icData)); } } int InstanceCount::getCount() { - std::lock_guard lg(_mx); - auto iter = _instances.find(_className); - if (iter == _instances.end()) { + std::lock_guard lg(_icData._mx); + auto iter = _icData._instances.find(_className); + if (iter == _icData._instances.end()) { return 0; } return iter->second; } std::ostream& operator<<(std::ostream& os, InstanceCount const& instanceCount) { - std::lock_guard lg(instanceCount._mx); - for (auto const& entry : instanceCount._instances) { + std::lock_guard lg(instanceCount._icData._mx); + for (auto const& entry : instanceCount._icData._instances) { if (entry.second != 0) { os << entry.first << "=" << entry.second << " "; } diff --git a/src/util/InstanceCount.h b/src/util/InstanceCount.h index 9923b84b5b..7e59d0dfea 100644 --- a/src/util/InstanceCount.h +++ b/src/util/InstanceCount.h @@ -4,6 +4,7 @@ #define LSST_QSERV_UTIL_INSTANCECOUNT_H // System headers +#include #include #include #include @@ -23,14 +24,26 @@ class InstanceCount { int getCount(); //< Return the number of instances of _className. + class InstanceCountData { + InstanceCountData(); + ~InstanceCountData(); + + friend InstanceCount; + friend std::ostream& operator<<(std::ostream& out, InstanceCount const& instanceCount); + + private: + std::map _instances; ///< Map of instances per class name. + std::recursive_mutex _mx; ///< Protects _instances. + std::atomic _instanceLogLimiter{0}; + }; + friend std::ostream& operator<<(std::ostream& out, InstanceCount const& instanceCount); private: - std::string _className; //< Names of the of which this is a member. - static std::map _instances; //< Map of instances per class name. - static std::recursive_mutex _mx; //< Protects _instances. - void _increment(std::string const& source); + + std::string _className; ///< Name of instance being counted. + static InstanceCountData _icData; ///< Map of counts and other data. }; } // namespace lsst::qserv::util diff --git a/src/util/MultiError.cc b/src/util/MultiError.cc index bf1ac13616..e4a5974571 100644 --- a/src/util/MultiError.cc +++ b/src/util/MultiError.cc @@ -34,16 +34,16 @@ using namespace std; namespace lsst::qserv::util { -std::string MultiError::toString() const { - std::ostringstream oss; +string MultiError::toString() const { + ostringstream oss; oss << *this; return oss.str(); } -std::string MultiError::toOneLineString() const { - std::ostringstream oss; - if (!this->empty()) { - if (this->size() > 1) { +string MultiError::toOneLineString() const { + ostringstream oss; + if (!empty()) { + if (size() > 1) { std::ostream_iterator string_it(oss, ", "); std::copy(_errorVector.begin(), _errorVector.end() - 1, string_it); } @@ -54,6 +54,15 @@ std::string MultiError::toOneLineString() const { int MultiError::firstErrorCode() const { return empty() ? ErrorCode::NONE : _errorVector.front().getCode(); } +string MultiError::firstErrorStr() const { + if (empty()) return string(); + ostringstream os; + os << _errorVector.front(); + return os.str(); +} + +util::Error MultiError::firstError() const { return empty() ? Error() : _errorVector.front(); } + bool MultiError::empty() const { return _errorVector.empty(); } std::vector::size_type MultiError::size() const { return _errorVector.size(); } diff --git a/src/util/MultiError.h b/src/util/MultiError.h index ea0d620488..2e22efb4cd 100644 --- a/src/util/MultiError.h +++ b/src/util/MultiError.h @@ -78,6 +78,10 @@ class MultiError : public std::exception { */ int firstErrorCode() const; + std::string firstErrorStr() const; + + util::Error firstError() const; + virtual ~MultiError() throw() {} /** Overload output operator for this class diff --git a/src/util/ThreadPool.h b/src/util/ThreadPool.h index c25bae28d6..8dd7a193a4 100644 --- a/src/util/ThreadPool.h +++ b/src/util/ThreadPool.h @@ -83,9 +83,9 @@ class CommandForThreadPool : public CommandTracked { public: using Ptr = std::shared_ptr; - CommandForThreadPool() = default; + CommandForThreadPool() : CommandTracked() {} explicit CommandForThreadPool(std::function func) : CommandTracked{func} {} - virtual ~CommandForThreadPool(); + ~CommandForThreadPool() override; /// Return true if the number of threads created and still existing is /// greater than the max. diff --git a/src/wbase/CMakeLists.txt b/src/wbase/CMakeLists.txt index 33ebbef0a0..1de19abbf5 100644 --- a/src/wbase/CMakeLists.txt +++ b/src/wbase/CMakeLists.txt @@ -14,12 +14,8 @@ install( TARGETS wbase ) -target_include_directories(wbase PRIVATE - ${XROOTD_INCLUDE_DIRS} -) target_link_libraries(wbase PUBLIC boost_regex log - XrdSsiLib ) diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index f286ac9760..171ed37448 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -365,7 +365,10 @@ void FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared } // Delete the result file as nobody will come looking for it. _kill(tMtxLock, " buildAndTransmitError"); - _uberJobData->responseError(multiErr, task->getChunkId(), cancelled, task->getLvlET()); + auto ujd = _uberJobData.lock(); + if (ujd != nullptr) { + ujd->responseError(multiErr, task->getChunkId(), cancelled, task->getLvlET()); + } } bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& task, @@ -433,16 +436,6 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetIdStr()); } - int const ujRowLimit = task->getRowLimit(); - bool rowLimitComplete = false; - if (ujRowLimit > 0 && _rowcount >= ujRowLimit) { - // There are enough rows to satisfy the query, so stop reading - hasMoreRows = false; - rowLimitComplete = true; - LOGS(_log, LOG_LVL_DEBUG, - __func__ << " enough rows for query rows=" << _rowcount << " " << task->getIdStr()); - } - // If no more rows are left in the task's result set then we need to check // if this is last task in a logical group of ones created for processing // the current request (note that certain classes of requests may require @@ -579,7 +572,10 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ // Prepare the response object and serialize in into a message that will // be sent to the Czar. string httpFileUrl = task->getUberJobData()->resultFileHttpUrl(); - _uberJobData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); + auto ujd = _uberJobData.lock(); + if (ujd != nullptr) { + ujd->responseFileReady(httpFileUrl, _rowcount, _transmitsize); + } return true; } diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index a9b5297db6..3ad2bdc8c8 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -236,7 +236,7 @@ class FileChannelShared { bool _isUberJob; ///< true if this is using UberJob http. To be removed when _sendChannel goes away. std::shared_ptr const _sendChannel; ///< Used to pass encoded information to XrdSsi. - std::shared_ptr _uberJobData; ///< Contains czar contact info. + std::weak_ptr _uberJobData; ///< Contains czar contact info. UberJobId const _uberJobId; ///< The UberJobId qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). TODO:UJ delete @@ -276,7 +276,6 @@ class FileChannelShared { int64_t _rowcount = 0; ///< The total numnber of rows in all result sets of a query. uint64_t _transmitsize = 0; ///< The total amount of data (bytes) in all result sets of a query. - uint64_t _headerCount = 0; ///< Count of headers received. /// _rowLimitComplete indicates that there is a LIMIT clause in the user query that /// can be applied to the queries given to workers. It's important to apply it diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index d4deea6097..5beff3c5a0 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -114,7 +114,9 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun vector const& fragSubTables, vector const& fragSubchunkIds, shared_ptr const& sc, std::shared_ptr const& queryStats_) - : _sendChannel(sc), + : _logLvlWT(LOG_LVL_WARN), + _logLvlET(LOG_LVL_ERROR), + _sendChannel(sc), _tSeq(++taskSequence), _qId(ujData->getQueryId()), _templateId(templateId), @@ -268,7 +270,6 @@ std::vector Task::createTasksFromUberJobMsg( return vect; } - std::vector Task::createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, std::shared_ptr const& sendChannel, int maxTableSizeMb, @@ -425,13 +426,19 @@ wpublish::QueryStatistics::Ptr Task::getQueryStats() const { } /// Flag the Task as cancelled, try to stop the SQL query, and try to remove it from the schedule. -void Task::cancel() { +void Task::cancel(bool logIt) { if (_cancelled.exchange(true)) { // Was already cancelled. return; } - LOGS(_log, LOG_LVL_DEBUG, "Task::cancel " << getIdStr()); + if (logIt) { + if (!_ujData->getCancelled()) { + LOGS(_log, LOG_LVL_DEBUG, "Task::cancel " << getIdStr() << " UberJob still live."); + } else { + LOGS(_log, LOG_LVL_TRACE, "Task::cancel " << getIdStr()); + } + } auto qr = _taskQueryRunner; // Need a copy in case _taskQueryRunner is reset. if (qr != nullptr) { qr->cancel(); @@ -452,13 +459,13 @@ bool Task::checkCancelled() { return _cancelled; } -/// @return true if task has already been cancelled. -bool Task::setTaskQueryRunner(TaskQueryRunner::Ptr const& taskQueryRunner) { +bool Task::setTaskQueryRunner(wdb::QueryRunner::Ptr const& taskQueryRunner) { _taskQueryRunner = taskQueryRunner; return checkCancelled(); } -void Task::freeTaskQueryRunner(TaskQueryRunner* tqr) { +void Task::freeTaskQueryRunner(wdb::QueryRunner* tqr) { + // Only free _taskQueryRunner if it's the expected one. if (_taskQueryRunner.get() == tqr) { _taskQueryRunner.reset(); } else { diff --git a/src/wbase/Task.h b/src/wbase/Task.h index cc5c7b5814..0f3a249a2f 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -46,6 +46,8 @@ #include "util/Histogram.h" #include "util/ThreadPool.h" +#include "util/InstanceCount.h" + // Forward declarations namespace lsst::qserv::mysql { class MySqlConfig; @@ -63,7 +65,8 @@ class SqlConnMgr; } namespace lsst::qserv::wdb { class ChunkResourceMgr; -} +class QueryRunner; +} // namespace lsst::qserv::wdb namespace lsst::qserv::wpublish { class QueriesAndChunks; class QueryStatistics; @@ -79,15 +82,6 @@ class TaskException : public util::Issue { explicit TaskException(util::Issue::Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} }; -/// Base class for tracking a database query for a worker Task. -class TaskQueryRunner { -public: - using Ptr = std::shared_ptr; - virtual ~TaskQueryRunner() {}; - virtual bool runQuery() = 0; - virtual void cancel() = 0; ///< Repeated calls to cancel() must be harmless. -}; - /// Class for storing database + table name. class TaskDbTbl { public: @@ -203,7 +197,7 @@ class Task : public util::CommandForThreadPool { /// This functional also attempts to inform the scheduler for this /// `Task` that is has been cancelled. The scheduler currently does /// nothing in this case. - void cancel(); + void cancel(bool logIt = true); /// Check if this task should be cancelled and call cancel() as needed. /// @return true if this task was or needed to be cancelled. @@ -211,9 +205,11 @@ class Task : public util::CommandForThreadPool { TaskState state() const { return _state; } std::string getQueryString() const; - bool setTaskQueryRunner( - TaskQueryRunner::Ptr const& taskQueryRunner); ///< return true if already cancelled. - void freeTaskQueryRunner(TaskQueryRunner* tqr); + /// Return true if already cancelled. + bool setTaskQueryRunner(std::shared_ptr const& taskQueryRunner); + + /// Free this instances TaskQueryRunner object, but only if the pointer matches `tqr` + void freeTaskQueryRunner(wdb::QueryRunner* tqr); void setTaskScheduler(TaskScheduler::Ptr const& scheduler) { _taskScheduler = scheduler; } TaskScheduler::Ptr getTaskScheduler() const { return _taskScheduler.lock(); } friend std::ostream& operator<<(std::ostream& os, Task const& t); @@ -338,7 +334,8 @@ class Task : public util::CommandForThreadPool { std::atomic _queryStarted{false}; ///< Set to true when the query is about to be run. std::atomic _cancelled{false}; - TaskQueryRunner::Ptr _taskQueryRunner; + std::atomic _safeToMoveRunning{false}; ///< false until done with waitForMemMan(). + std::shared_ptr _taskQueryRunner; std::weak_ptr _taskScheduler; bool _onInteractive{ false}; ///< True if the scheduler put this task on the interactive (group) scheduler. diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index fc17dc248d..07a7df6673 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -38,6 +38,7 @@ #include "http/Method.h" #include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" +#include "protojson/JobReadyMsg.h" #include "util/Bug.h" #include "util/MultiError.h" #include "wconfig/WorkerConfig.h" @@ -87,11 +88,9 @@ void UberJobData::setFileChannelShared(std::shared_ptr const& _fileChannelShared = fileChannelShared; } -void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, - uint64_t headerCount) { +void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize) { LOGS(_log, LOG_LVL_INFO, - cName(__func__) << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize - << " headerCount=" << headerCount); + cName(__func__) << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize); // Latch to prevent errors from being transmitted. // NOTE: Calls to responseError() and responseFileReady() are protected by the @@ -100,14 +99,6 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _responseState was " << _responseState << " instead of NOTHING"); } - string workerIdStr; - if (_foreman != nullptr) { - workerIdStr = _foreman->chunkInventory()->id(); - } else { - workerIdStr = "dummyWorkerIdStr"; - LOGS(_log, LOG_LVL_INFO, - cName(__func__) << " _foreman was null, which should only happen in unit tests"); - } string workerIdStr; if (_foreman != nullptr) { @@ -118,17 +109,11 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount cName(__func__) << " _foreman was null, which should only happen in unit tests"); } - json request = {{"version", http::MetaModule::version}, - {"workerid", workerIdStr}, - {"auth_key", _authKey}, - {"czar", _czarName}, - {"czarid", _czarId}, - {"queryid", _queryId}, - {"uberjobid", _uberJobId}, - {"fileUrl", httpFileUrl}, - {"rowCount", rowCount}, - {"fileSize", fileSize}, - {"headerCount", headerCount}}; + auto repliInstId = wconfig::WorkerConfig::instance()->replicationInstanceId(); + auto repliAuthKey = wconfig::WorkerConfig::instance()->replicationAuthKey(); + auto jrMsg = protojson::JobReadyMsg::create(repliInstId, repliAuthKey, workerIdStr, _czarName, _czarId, + _queryId, _uberJobId, httpFileUrl, rowCount, fileSize); + json request = jrMsg->serializeJson(); auto const method = http::Method::POST; vector const headers = {"Content-Type: application/json"}; @@ -219,23 +204,23 @@ string UberJobData::resultFilePath() const { } std::string UberJobData::resultFileHttpUrl() const { - auto const workerConfig = wconfig::WorkerConfig::instance(); - auto const resultDeliveryProtocol = workerConfig->resultDeliveryProtocol(); - if (resultDeliveryProtocol != wconfig::ConfigValResultDeliveryProtocol::HTTP) { - throw runtime_error("wbase::Task::Task: unsupported results delivery protocol: " + - wconfig::ConfigValResultDeliveryProtocol::toString(resultDeliveryProtocol)); - } // TODO:UJ it seems like this should just be part of the FileChannelShared??? return "http://" + _foreman->getFqdn() + ":" + to_string(_resultsHttpPort) + "/" + _resultFileName(); } void UberJobData::cancelAllTasks() { LOGS(_log, LOG_LVL_INFO, cName(__func__)); + int count = 0; if (_cancelled.exchange(true) == false) { lock_guard lg(_ujTasksMtx); for (auto const& task : _ujTasks) { - task->cancel(); + auto tsk = task.lock(); + if (tsk != nullptr) { + tsk->cancel(false); + ++count; + } } + LOGS(_log, LOG_LVL_INFO, cName(__func__) << " cancelled " << count << " Tasks"); } } @@ -335,109 +320,4 @@ UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { return newPtr; } -void UberJobData::cancelAllTasks() { - LOGS(_log, LOG_LVL_INFO, cName(__func__)); - if (_cancelled.exchange(true) == false) { - lock_guard lg(_ujTasksMtx); - for (auto const& task : _ujTasks) { - task->cancel(); - } - } -} - -string UJTransmitCmd::cName(const char* funcN) const { - stringstream os; - os << "UJTransmitCmd::" << funcN << " czId=" << _czarId << " QID=" << _queryId << "_ujId=" << _uberJobId; - return os.str(); -} - -void UJTransmitCmd::action(util::CmdData* data) { - LOGS(_log, LOG_LVL_TRACE, cName(__func__)); - // Make certain _selfPtr is reset before leaving this function. - // If a retry is needed, duplicate() is called. - class ResetSelf { - public: - ResetSelf(UJTransmitCmd* ujtCmd) : _ujtCmd(ujtCmd) {} - ~ResetSelf() { _ujtCmd->_selfPtr.reset(); } - UJTransmitCmd* const _ujtCmd; - }; - ResetSelf resetSelf(this); - - _attemptCount++; - auto ujPtr = _ujData.lock(); - if (ujPtr == nullptr || ujPtr->getCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " UberJob was cancelled " << _attemptCount); - return; - } - http::Client client(_method, _url, _requestStr, _headers); - bool transmitSuccess = false; - try { - json const response = client.readAsJson(); - if (0 != response.at("success").get()) { - transmitSuccess = true; - } else { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " Transmit success == 0"); - // There's no point in re-sending as the czar got the message and didn't like - // it. - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) + " " + _requestContext + " failed, ex: " + ex.what()); - } - - if (!transmitSuccess) { - auto sPtr = _selfPtr; - if (_foreman != nullptr && sPtr != nullptr) { - // Do not reset _selfPtr as re-queuing may be needed several times. - LOGS(_log, LOG_LVL_WARN, - cName(__func__) << " no response for transmit, putting on failed transmit queue."); - auto wCzInfo = _foreman->getWCzarInfoMap()->getWCzarInfo(_czarId); - // This will check if the czar is believed to be alive and try the queue the query to be tried - // again at a lower priority. It it thinks the czar is dead, it will throw it away. - // TODO:UJ I have my doubts about this as a reconnected czar may go down in flames - // as it is hit with thousands of these. - // Alternate plan, set a flag in the status message response (WorkerQueryStatusData) - // indicates some messages failed. When the czar sees the flag, it'll request a - // message from the worker that contains all of the failed transmit data and handle - // that. All of these failed transmits should fit in a single message. - if (wCzInfo->checkAlive(CLOCK::now())) { - auto wPool = _foreman->getWPool(); - if (wPool != nullptr) { - Ptr replacement = duplicate(); - if (replacement != nullptr) { - wPool->queCmd(replacement, 2); - } else { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " replacement was null"); - } - } else { - // No thread pool, should only be possible in unit tests. - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " no wPool"); - return; - } - } - } else { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " _selfPtr was null, assuming job killed."); - } - } -} - -void UJTransmitCmd::kill() { - LOGS(_log, LOG_LVL_WARN, cName(__func__)); - auto sPtr = _selfPtr; - _selfPtr.reset(); - if (sPtr == nullptr) { - return; - } -} - -UJTransmitCmd::Ptr UJTransmitCmd::duplicate() { - LOGS(_log, LOG_LVL_INFO, cName(__func__)); - auto ujD = _ujData.lock(); - if (ujD == nullptr) { - return nullptr; - } - Ptr newPtr = create(_foreman, ujD, _method, _headers, _url, _requestContext, _requestStr); - newPtr->_attemptCount = _attemptCount; - return newPtr; -} - } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index 5922edeeaf..9c5cd4844e 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -39,6 +39,8 @@ #include "util/QdispPool.h" #include "wbase/SendChannel.h" +#include "util/InstanceCount.h" + namespace lsst::qserv { namespace protojson { @@ -101,11 +103,10 @@ class UberJobData : public std::enable_shared_from_this { } /// Let the czar know the result is ready. - void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, - uint64_t headerCount); // TODO:UJ remove headerCount + void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize); /// Let the Czar know there's been a problem. - bool responseError(util::MultiError& multiErr, int chunkId, bool cancelled); + void responseError(util::MultiError& multiErr, int chunkId, bool cancelled, int logLvl); std::string const& getIdStr() const { return _idStr; } std::string cName(std::string const& funcName) { return "UberJobData::" + funcName + " " + getIdStr(); } @@ -119,11 +120,10 @@ class UberJobData : public std::enable_shared_from_this { /// that there is no limit to the number of rows sent back by the worker. /// Workers can only safely limit rows for queries that have the LIMIT clause without other related /// clauses like ORDER BY. - int getRowLimit() { return _rowLimit; } + int getRowLimit() const { return _rowLimit; } - std::string buildUjResultFilePath(std::string const& resultsDirname); - std::string resultFilePath(); - std::string resultFileHttpUrl(); + std::string resultFilePath() const; + std::string resultFileHttpUrl() const; private: UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, @@ -154,7 +154,7 @@ class UberJobData : public std::enable_shared_from_this { std::shared_ptr const _foreman; - std::vector> _ujTasks; + std::vector> _ujTasks; std::shared_ptr _fileChannelShared; std::mutex _ujTasksMtx; ///< Protects _ujTasks. diff --git a/src/wbase/UserQueryInfo.h b/src/wbase/UserQueryInfo.h index eb15de7089..2a4f701d13 100644 --- a/src/wbase/UserQueryInfo.h +++ b/src/wbase/UserQueryInfo.h @@ -32,6 +32,7 @@ // Qserv headers #include "global/intTypes.h" +#include "util/InstanceCount.h" // This header declarations namespace lsst::qserv::wbase { @@ -93,6 +94,7 @@ class UserQueryInfo { private: UserQueryInfo(QueryId qId, CzarIdType czId); + util::InstanceCount const _icUqi{"UserQueryInfo"}; QueryId const _qId; ///< The User Query Id number. CzarIdType const _czarId; diff --git a/src/wconfig/CMakeLists.txt b/src/wconfig/CMakeLists.txt index 4fef7302c0..bc667a7ea1 100644 --- a/src/wconfig/CMakeLists.txt +++ b/src/wconfig/CMakeLists.txt @@ -23,6 +23,3 @@ target_link_libraries(testSanityCheck add_test(NAME testSanityCheck COMMAND testSanityCheck) -install( - TARGETS wconfig -) diff --git a/src/wconfig/WorkerConfig.h b/src/wconfig/WorkerConfig.h index 117580558e..e4896c5a00 100644 --- a/src/wconfig/WorkerConfig.h +++ b/src/wconfig/WorkerConfig.h @@ -435,7 +435,6 @@ class WorkerConfig { util::ConfigValTUInt::create(_configValMap, "czar", "DeadTimeSec", notReq, 180); CVTUIntPtr _czarComNumHttpThreads = util::ConfigValTUInt::create(_configValMap, "czar", "ComNumHttpThreads", notReq, 40); - }; } // namespace lsst::qserv::wconfig diff --git a/src/wcontrol/CMakeLists.txt b/src/wcontrol/CMakeLists.txt index ec8729388b..9f447ed0a2 100644 --- a/src/wcontrol/CMakeLists.txt +++ b/src/wcontrol/CMakeLists.txt @@ -13,13 +13,9 @@ install( TARGETS wcontrol ) -target_include_directories(wcontrol PRIVATE - ${XROOTD_INCLUDE_DIRS} -) target_link_libraries(wcontrol PUBLIC log - XrdSsiLib qhttp wdb ) diff --git a/src/wcontrol/WCzarInfoMap.cc b/src/wcontrol/WCzarInfoMap.cc index d1554de3f4..0ccef5a787 100644 --- a/src/wcontrol/WCzarInfoMap.cc +++ b/src/wcontrol/WCzarInfoMap.cc @@ -31,6 +31,7 @@ // qserv headers #include "http/Client.h" +#include "protojson/WorkerCzarComIssue.h" #include "protojson/WorkerQueryStatusData.h" #include "util/Bug.h" #include "util/Histogram.h" diff --git a/src/wdb/CMakeLists.txt b/src/wdb/CMakeLists.txt index bbf9340312..21546e1daa 100644 --- a/src/wdb/CMakeLists.txt +++ b/src/wdb/CMakeLists.txt @@ -7,13 +7,13 @@ target_sources(wdb PRIVATE SQLBackend.cc ) -target_include_directories(wdb PRIVATE - ${XROOTD_INCLUDE_DIRS} -) target_link_libraries(wdb PUBLIC log - XrdSsiLib +) + +install( + TARGETS wdb ) install( @@ -23,9 +23,6 @@ install( FUNCTION(wdb_tests) FOREACH(TEST IN ITEMS ${ARGV}) add_executable(${TEST} ${TEST}.cc) - target_include_directories(${TEST} PRIVATE - ${XROOTD_INCLUDE_DIRS} - ) target_link_libraries(${TEST} PUBLIC crypto xrdsvc diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index a2e381ffae..e497aaa0db 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -136,7 +136,7 @@ bool QueryRunner::runQuery() { // Make certain our Task knows that this object is no longer in use when this function exits. class Release { public: - Release(wbase::Task::Ptr t, wbase::TaskQueryRunner* tqr, + Release(wbase::Task::Ptr t, QueryRunner* tqr, shared_ptr const& queriesAndChunks) : _t{t}, _tqr{tqr}, _queriesAndChunks(queriesAndChunks) {} ~Release() { @@ -146,7 +146,7 @@ bool QueryRunner::runQuery() { private: wbase::Task::Ptr _t; - wbase::TaskQueryRunner* _tqr; + QueryRunner* _tqr; shared_ptr const _queriesAndChunks; }; Release release(_task, this, _queriesAndChunks); @@ -156,6 +156,7 @@ bool QueryRunner::runQuery() { return false; } + LOGS(_log, LOG_LVL_TRACE, "QR in flight for sqlConnMgr " << _sqlConnMgr->dump()); // Queries that span multiple tasks should not be high priority for the SqlConMgr as it risks deadlock. bool interactive = _task->getScanInteractive() && !(_task->getSendChannel()->getTaskCount() > 1); wcontrol::SqlConnLock sqlConnLock(*_sqlConnMgr, not interactive, _task->getSendChannel()); diff --git a/src/wdb/QueryRunner.h b/src/wdb/QueryRunner.h index 6d5236d95d..0370cab501 100644 --- a/src/wdb/QueryRunner.h +++ b/src/wdb/QueryRunner.h @@ -58,7 +58,7 @@ namespace lsst::qserv::wdb { /// On the worker, run a query related to a Task, hold the resources needed to run the query, /// and write the results to the supplied SendChannel. /// -class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_from_this { +class QueryRunner : public std::enable_shared_from_this { public: using Ptr = std::shared_ptr; static QueryRunner::Ptr newQueryRunner( @@ -70,13 +70,14 @@ class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_fro QueryRunner& operator=(QueryRunner const&) = delete; virtual ~QueryRunner() = default; - bool runQuery() override; + bool runQuery(); /// Cancel the action (in-progress). This should only be called /// by Task::cancel(), so if this needs to be cancelled elsewhere, /// call Task::cancel(). /// This should kill an in progress SQL command. - void cancel() override; + /// Repeated calls to cancel() must be harmless. + void cancel(); protected: QueryRunner(wbase::Task::Ptr const& task, ChunkResourceMgr::Ptr const& chunkResourceMgr, @@ -86,6 +87,7 @@ class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_fro private: bool _initConnection(); + void _setDb(); /// Dispatch with output sent through a SendChannel bool _dispatchChannel(); @@ -95,6 +97,7 @@ class QueryRunner : public wbase::TaskQueryRunner, public std::enable_shared_fro /// Resource reservation ChunkResourceMgr::Ptr _chunkResourceMgr; + std::atomic _cancelled{false}; mysql::MySqlConfig const _mySqlConfig; std::unique_ptr _mysqlConn; diff --git a/src/wpublish/CMakeLists.txt b/src/wpublish/CMakeLists.txt index 47ac6a8512..946b0c665a 100644 --- a/src/wpublish/CMakeLists.txt +++ b/src/wpublish/CMakeLists.txt @@ -7,10 +7,6 @@ target_sources(wpublish PRIVATE QueryStatistics.cc ) -target_include_directories(wpublish PRIVATE - ${XROOTD_INCLUDE_DIRS} -) - install( TARGETS wpublish ) @@ -19,8 +15,6 @@ target_link_libraries(wpublish PUBLIC log proto protobuf - XrdSsiLib - XrdCl ) add_executable(testChunkInventory testChunkInventory.cc) @@ -32,7 +26,3 @@ target_link_libraries(testChunkInventory PUBLIC ) add_test(NAME testChunkInventory COMMAND testChunkInventory) - -install( - TARGETS wpublish -) diff --git a/src/wpublish/QueriesAndChunks.h b/src/wpublish/QueriesAndChunks.h index 757267fbc7..a3d89e1e64 100644 --- a/src/wpublish/QueriesAndChunks.h +++ b/src/wpublish/QueriesAndChunks.h @@ -202,13 +202,6 @@ class QueriesAndChunks { /// @see _addQueryId QueryStatistics::Ptr addQueryId(QueryId qId, CzarIdType czarId); - /// Return the statistics for a user query, creating if needed. - /// Since it is possible to get messages out of order, there - /// are several case where something like a cancellation - /// message arrives before any tasks have been created. - /// @see getStats() - QueryStatistics::Ptr addQueryId(QueryId qId, CzarIdType czarId); - void addTask(wbase::Task::Ptr const& task); void addTasks(std::vector const& tasks, std::vector& cmds); void queuedTask(wbase::Task::Ptr const& task); diff --git a/src/wpublish/QueryStatistics.h b/src/wpublish/QueryStatistics.h index 5fd24ff003..2f1eea8736 100644 --- a/src/wpublish/QueryStatistics.h +++ b/src/wpublish/QueryStatistics.h @@ -39,6 +39,7 @@ // Qserv headers #include "global/intTypes.h" +#include "util/InstanceCount.h" #include "wbase/Task.h" #include "wsched/SchedulerBase.h" @@ -172,6 +173,7 @@ class QueryStatistics { explicit QueryStatistics(QueryId queryId, CzarIdType czarId); bool _isMostlyDead() const; + util::InstanceCount const _icqs{"QueryStatistics"}; mutable std::mutex _qStatsMtx; std::chrono::system_clock::time_point _touched = std::chrono::system_clock::now(); diff --git a/src/wsched/CMakeLists.txt b/src/wsched/CMakeLists.txt index c05eb03ba3..e3cab0d3f6 100644 --- a/src/wsched/CMakeLists.txt +++ b/src/wsched/CMakeLists.txt @@ -9,6 +9,10 @@ target_sources(wsched PRIVATE SchedulerBase.cc ) +install( + TARGETS wsched +) + target_link_libraries(wsched PUBLIC log ) @@ -19,10 +23,6 @@ install( add_executable(testSchedulers testSchedulers.cc) -target_include_directories(testSchedulers PRIVATE - ${XROOTD_INCLUDE_DIRS} -) - target_link_libraries(testSchedulers PUBLIC wsched xrdsvc diff --git a/src/wsched/testSchedulers.cc b/src/wsched/testSchedulers.cc index 17ff5523c9..f2217b1b68 100644 --- a/src/wsched/testSchedulers.cc +++ b/src/wsched/testSchedulers.cc @@ -303,7 +303,6 @@ BOOST_AUTO_TEST_CASE(Grouping) { } BOOST_AUTO_TEST_CASE(GroupMaxThread) { - #if 0 // &&& fix and re-enable // Test that maxThreads is meaningful. LOGS(_log, LOG_LVL_WARN, "Test_case GroupMaxThread"); diff --git a/src/xrdsvc/CMakeLists.txt b/src/xrdsvc/CMakeLists.txt index 818455ad75..fd90aee825 100644 --- a/src/xrdsvc/CMakeLists.txt +++ b/src/xrdsvc/CMakeLists.txt @@ -1,12 +1,11 @@ add_library(xrdsvc SHARED) -add_dependencies(xrdsvc proto) -target_sources(qserv_xrdsvc PRIVATE +target_sources(xrdsvc PRIVATE HttpModule.cc HttpMonitorModule.cc HttpReplicaMgtModule.cc - HttpWorkerCzarModule.cc HttpSvc.cc + HttpWorkerCzarModule.cc SsiProvider.cc SsiService.cc ) @@ -19,13 +18,15 @@ target_link_libraries(xrdsvc PUBLIC boost_filesystem boost_system global + http log mysql - http + protojson sql wbase wconfig wcontrol + wdb wpublish wsched XrdSsiLib diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index c42321384a..b9446ec62d 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -248,13 +248,6 @@ json HttpWorkerCzarModule::_handleQueryStatus(std::string const& func) { wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czId); wCzarInfo->czarMsgReceived(CLOCK::now()); - auto const czInfo = wqsData->getCzInfo(); - LOGS(_log, LOG_LVL_TRACE, " HttpWorkerCzarModule::_handleQueryStatus req=" << jsReq.dump()); - CzarIdType czId = czInfo->czId; - wcontrol::WCzarInfoMap::Ptr wCzarMap = foreman()->getWCzarInfoMap(); - wcontrol::WCzarInfo::Ptr wCzarInfo = wCzarMap->getWCzarInfo(czId); - wCzarInfo->czarMsgReceived(CLOCK::now()); - // For all queryId and czarId items, if the item can't be found, it is simply ignored. Anything that // is missed will eventually be picked up by other mechanisms, such as results being rejected // by the czar. This almost never happen, but the system should respond gracefully. From b5aa5dac420b5fd8286ba5d5a3d3a9a7e7ef1517 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 29 Apr 2025 08:25:32 -0700 Subject: [PATCH 10/15] Added JobErrorMsg. --- src/ccontrol/MergingHandler.cc | 366 ++++++++++++------ src/ccontrol/MergingHandler.h | 12 +- src/ccontrol/UserQueryFactory.cc | 2 +- src/ccontrol/UserQueryProcessList.cc | 7 +- src/ccontrol/UserQueryResources.cc | 1 + src/ccontrol/UserQuerySelect.cc | 6 +- src/czar/HttpCzarWorkerModule.cc | 28 +- src/mysql/CMakeLists.txt | 2 +- src/mysql/LocalInfile.cc | 17 +- src/mysql/LocalInfile.h | 4 +- src/protojson/CMakeLists.txt | 8 +- src/protojson/JobReadyMsg.cc | 116 ------ src/protojson/UberJobErrorMsg.cc | 124 ++++++ .../{JobReadyMsg.h => UberJobErrorMsg.h} | 71 ++-- src/protojson/UberJobReadyMsg.cc | 127 ++++++ src/protojson/UberJobReadyMsg.h | 99 +++++ src/protojson/WorkerCzarComIssue.cc | 4 +- src/protojson/WorkerCzarComIssue.h | 6 +- src/protojson/testStatusData.cc | 4 +- src/protojson/testUberJobErrorMsg.cc | 106 +++++ ...tJobReadyMsg.cc => testUberJobReadyMsg.cc} | 19 +- src/qdisp/ResponseHandler.h | 8 +- src/qdisp/UberJob.cc | 10 +- src/qdisp/testQDisp.cc | 3 +- src/rproc/InfileMerger.cc | 36 +- src/rproc/InfileMerger.h | 10 +- src/sql/SqlResults.cc | 21 + src/sql/SqlResults.h | 14 +- src/wbase/FileChannelShared.cc | 31 +- src/wbase/FileChannelShared.h | 3 +- src/wbase/UberJobData.cc | 38 +- src/wcontrol/WCzarInfoMap.cc | 2 +- src/wdb/QueryRunner.cc | 3 +- src/wdb/testQueryRunner.cc | 1 + src/wsched/ScanScheduler.cc | 1 - src/xrdsvc/SsiService.cc | 31 +- 36 files changed, 905 insertions(+), 436 deletions(-) delete mode 100644 src/protojson/JobReadyMsg.cc create mode 100644 src/protojson/UberJobErrorMsg.cc rename src/protojson/{JobReadyMsg.h => UberJobErrorMsg.h} (52%) create mode 100644 src/protojson/UberJobReadyMsg.cc create mode 100644 src/protojson/UberJobReadyMsg.h create mode 100644 src/protojson/testUberJobErrorMsg.cc rename src/protojson/{testJobReadyMsg.cc => testUberJobReadyMsg.cc} (82%) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 8bd43e7381..66df1aeff1 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -87,16 +87,10 @@ lsst::qserv::TimeCountTracker::CALLBACKFUNC const reportFileRecvRate = }; -/// If success, then everything is fine. -/// If not success, and not mergeHappened, the user query can be saved by abandoning -/// this UberJob. If mergeHappened, the result table is fouled and the user query is ruined. -/// @return bool success - true if operation was successful. -/// @return bool mergeHappened - true if merging was started. -std::tuple readHttpFileAndMergeHttp( - lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, - function const& messageIsReady, - shared_ptr const& httpConnPool) { - string const context = "MergingHandler::" + string(__func__) + " " + " qid=" + uberJob->getIdStr() + " "; +string readHttpFileAndMerge(lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, + size_t fileSize, function const& messageIsReady, + shared_ptr const& httpConnPool) { + string const context = "MergingHandler::" + string(__func__) + " "; LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); @@ -108,26 +102,8 @@ std::tuple readHttpFileAndMergeHttp( // A location of the next byte to be read from the input file. The variable // is used for error reporting. - uint64_t offset = 0; - - // Temporary buffer for messages read from the file. The buffer gets automatically - // resized to fit the largest message. - unique_ptr msgBuf; - size_t msgBufSize = 0; - size_t msgBufNext = 0; // An index of the next character in the buffer. - - // Fixed-size buffer to store the message size. - string msgSizeBuf(sizeof(uint32_t), '\0'); - size_t msgSizeBufNext = 0; // An index of the next character in the buffer. - - // The size of the next/current message. The variable is set after succesfully parsing - // the message length header and is reset back to 0 after parsing the message body. - // The value is stays 0 while reading the frame header. - uint32_t msgSizeBytes = 0; - bool success = true; - bool mergeHappened = false; - uint64_t headerCount = 0; - uint64_t totalBytesRead = 0; + size_t offset = 0; + try { auto exec = uberJob->getExecutive(); if (exec == nullptr || exec->getCancelled()) { @@ -143,105 +119,132 @@ std::tuple readHttpFileAndMergeHttp( clientConfig.tcpKeepIntvl = 5; // the default is 60 sec http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, httpConnPool); + + // Starts the tracker to measure the performance of the network I/O. + transmitRateTracker = make_unique>(reportFileRecvRate); + + // Start reading the file. The read() method will call the callback function + // for each chunk of data read from the file. reader.read([&](char const* inBuf, size_t inBufSize) { - // A value of the flag is set by the message processor when it's time to finish - // or abort reading the file. - bool last = false; - char const* next = inBuf; - char const* const end = inBuf + inBufSize; - LOGS(_log, LOG_LVL_TRACE, context << " next=" << (uint64_t)next << " end=" << (uint64_t)end); - while ((next < end) && !last) { - if (exec->getCancelled()) { - throw runtime_error(context + " query was cancelled"); - } - if (msgSizeBytes == 0) { - // Continue or finish reading the frame header. - size_t const bytes2read = - std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); - std::memcpy(msgSizeBuf.data() + msgSizeBufNext, next, bytes2read); - next += bytes2read; - offset += bytes2read; - msgSizeBufNext += bytes2read; - if (msgSizeBufNext == sizeof(uint32_t)) { - ++headerCount; - // Done reading the frame header. - msgSizeBufNext = 0; - // Parse and evaluate the message length. - msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); - if (msgSizeBytes == 0) { - throw runtime_error("message size is 0 at offset " + - to_string(offset - sizeof(uint32_t)) + ", file: " + httpUrl); - } - if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { - throw runtime_error("message size " + to_string(msgSizeBytes) + " at offset " + - to_string(offset - sizeof(uint32_t)) + - " exceeds the hard limit of " + - to_string(ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) + - ", file: " + httpUrl); - } - // Extend the message buffer (if needed). Note that buffer never gets - // truncated to avoid excessive memory deallocations/allocations. - if (msgBufSize < msgSizeBytes) { - msgBufSize = msgSizeBytes; - msgBuf.reset(new char[msgBufSize]); - } - // Starts the tracker to measure the performance of the network I/O. - transmitRateTracker = - make_unique>(reportFileRecvRate); + // Check if the end of the file has been reached. + // Destroying the tracker will result in stopping the tracker's timer and + // reporting the file read rate before proceeding to the merge. + transmitRateTracker->addToValue(inBufSize); + transmitRateTracker->setSuccess(); + transmitRateTracker.reset(); + messageIsReady(inBuf, inBufSize); + offset += inBufSize; + // Restart the tracker to measure the reading performance of the next chunk of data. + transmitRateTracker = make_unique>(reportFileRecvRate); + }); + if (offset != fileSize) { + throw runtime_error(context + "short read"); + } + } catch (exception const& ex) { + string const errMsg = "failed to open/read: " + httpUrl + ", fileSize: " + to_string(fileSize) + + ", offset: " + to_string(offset) + ", ex: " + string(ex.what()); + LOGS(_log, LOG_LVL_ERROR, context << errMsg); + return errMsg; + /* &&& + // A value of the flag is set by the message processor when it's time to finish + // or abort reading the file. + bool last = false; + char const* next = inBuf; + char const* const end = inBuf + inBufSize; + LOGS(_log, LOG_LVL_TRACE, context << " next=" << (uint64_t)next << " end=" << (uint64_t)end); + while ((next < end) && !last) { + if (exec->getCancelled()) { + throw runtime_error(context + " query was cancelled"); } - } else { - // Continue or finish reading the message body. - size_t const bytes2read = - std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); - std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); - next += bytes2read; - offset += bytes2read; - msgBufNext += bytes2read; - if (msgBufNext == msgSizeBytes) { - // Done reading message body. - msgBufNext = 0; - - // Destroying the tracker will result in stopping the tracker's timer and - // reporting the file read rate before proceeding to the merge. - if (transmitRateTracker != nullptr) { - transmitRateTracker->addToValue(msgSizeBytes); - transmitRateTracker->setSuccess(); - transmitRateTracker.reset(); - } - - // Parse and evaluate the message. - mergeHappened = true; - bool messageReadyResult = messageIsReady(msgBuf.get(), msgSizeBytes, last); - totalBytesRead += msgSizeBytes; - if (!messageReadyResult) { - success = false; - throw runtime_error("message processing failed at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + if (msgSizeBytes == 0) { + // Continue or finish reading the frame header. + size_t const bytes2read = + std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); + std::memcpy(msgSizeBuf.data() + msgSizeBufNext, next, bytes2read); + next += bytes2read; + offset += bytes2read; + msgSizeBufNext += bytes2read; + if (msgSizeBufNext == sizeof(uint32_t)) { + ++headerCount; + // Done reading the frame header. + msgSizeBufNext = 0; + // Parse and evaluate the message length. + msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); + if (msgSizeBytes == 0) { + throw runtime_error("message size is 0 at offset " + + to_string(offset - sizeof(uint32_t)) + ", file: " + + httpUrl); + } + if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { + throw runtime_error("message size " + to_string(msgSizeBytes) + " at offset " + + to_string(offset - sizeof(uint32_t)) + " exceeds the hard limit of " + + to_string(ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) + + ", file: " + httpUrl); + } + // Extend the message buffer (if needed). Note that buffer never gets + // truncated to avoid excessive memory deallocations/allocations. + if (msgBufSize < msgSizeBytes) { + msgBufSize = msgSizeBytes; + msgBuf.reset(new char[msgBufSize]); + } + // Starts the tracker to measure the performance of the network I/O. + transmitRateTracker = + make_unique>(reportFileRecvRate); } - // Reset the variable to prepare for reading the next header & message (if any). - msgSizeBytes = 0; } else { - LOGS(_log, LOG_LVL_TRACE, - context << " headerCount=" << headerCount - << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); + // Continue or finish reading the message body. + size_t const bytes2read = + std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); + std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); + next += bytes2read; + offset += bytes2read; + msgBufNext += bytes2read; + if (msgBufNext == msgSizeBytes) { + // Done reading message body. + msgBufNext = 0; + + // Destroying the tracker will result in stopping the tracker's timer and + // reporting the file read rate before proceeding to the merge. + if (transmitRateTracker != nullptr) { + transmitRateTracker->addToValue(msgSizeBytes); + transmitRateTracker->setSuccess(); + transmitRateTracker.reset(); + } + + // Parse and evaluate the message. + mergeHappened = true; + bool messageReadyResult = messageIsReady(msgBuf.get(), msgSizeBytes, last); + totalBytesRead += msgSizeBytes; + if (!messageReadyResult) { + success = false; + throw runtime_error("message processing failed at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + // Reset the variable to prepare for reading the next header & message (if any). + msgSizeBytes = 0; + } else { + LOGS(_log, LOG_LVL_TRACE, + context << " headerCount=" << headerCount + << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); + } } } + }); + LOGS(_log, LOG_LVL_TRACE, + context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes + << " totalBytesRead=" << totalBytesRead); + if (msgSizeBufNext != 0) { + throw runtime_error("short read of the message header at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); } - }); - LOGS(_log, LOG_LVL_TRACE, - context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes - << " totalBytesRead=" << totalBytesRead); - if (msgSizeBufNext != 0) { - throw runtime_error("short read of the message header at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - if (msgBufNext != 0) { - throw runtime_error("short read of the message body at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); - success = false; + if (msgBufNext != 0) { + throw runtime_error("short read of the message body at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); + success = false; + */ } // Remove the file from the worker if it still exists. Report and ignore errors. @@ -252,10 +255,8 @@ std::tuple readHttpFileAndMergeHttp( } catch (exception const& ex) { LOGS(_log, LOG_LVL_WARN, context << "failed to remove " << httpUrl << ", ex: " << ex.what()); } - // If the merge failed, that indicates something went wrong in the local database table, - // is likely this user query is doomed and should be cancelled. - LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeHappened); - return {success, mergeHappened}; + + return string(); } } // namespace @@ -336,6 +337,118 @@ std::ostream& MergingHandler::print(std::ostream& os) const { return os << "MergingRequester(flushed=" << (_flushed ? "true)" : "false)"); } +bool queryIsNoLongerActive(qdisp::UberJob::Ptr const& uberJob) { //&&& + // Do nothing if the query got cancelled for any reason. + if (uberJob->isQueryCancelled()) return true; + + // Check for other indicators that the query may have cancelled or finished. + auto executive = uberJob->getExecutive(); + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { + return true; + } + return false; +} + +bool MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uberJob, string const& fileUrl, + uint64_t fileSize) { + if (_flushed) { + throw util::Bug(ERR_LOC, "already flushed"); + } + + if (fileSize == 0) return true; + + // After this final test the job's result processing can't be interrupted. + if (uberJob->isQueryCancelled()) return true; + + // Read from the http stream and push records into the CSV stream in a separate thread. + // Note the fixed capacity of the stream which allows up to 2 records to be buffered + // in the stream. This is enough to hide the latency of the HTTP connection and + // the time needed to read the file. + auto csvStream = mysql::CsvStream::create(2); + string fileReadErrorMsg; + thread csvThread([uberJob, csvStream, fileUrl, fileSize, &fileReadErrorMsg]() { + size_t bytesRead = 0; + fileReadErrorMsg = ::readHttpFileAndMerge( + uberJob, fileUrl, fileSize, + [uberJob, csvStream, fileSize, &bytesRead](char const* buf, uint32_t size) { + bool last = false; + if (buf == nullptr || size == 0) { + last = true; + } else { + csvStream->push(buf, size); + bytesRead += size; + last = bytesRead >= fileSize; + } + if (last) { + csvStream->push(nullptr, 0); + } + }, + MergingHandler::_getHttpConnPool()); + // Push the stream terminator to indicate the end of the stream. + // It may be neeeded to unblock the table merger which may be still attempting to read + // from the CSV stream. + if (!fileReadErrorMsg.empty()) { + csvStream->push(nullptr, 0); + } + }); + + // Attempt the actual merge. + bool const fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvStream); + if (!fileMergeSuccess) { + LOGS(_log, LOG_LVL_WARN, __func__ << " merge failed"); + util::Error const& err = _infileMerger->getError(); + _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg(), util::ErrorCode::RESULT_IMPORT); + } + + csvThread.join(); + if (!fileReadErrorMsg.empty()) { + LOGS(_log, LOG_LVL_WARN, __func__ << " result file read failed"); + _setError(ccontrol::MSG_HTTP_RESULT, fileReadErrorMsg, util::ErrorCode::RESULT_IMPORT); + } + _flushed = true; + return fileMergeSuccess && fileReadErrorMsg.empty(); +} + +void MergingHandler::_setError(int code, std::string const& msg, int errorState) { + LOGS(_log, LOG_LVL_DEBUG, "_setError: code: " << code << ", message: " << msg); + auto exec = _executive.lock(); + if (exec == nullptr) return; + exec->addMultiError(code, msg, errorState); +} + +tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t fileSize, uint64_t expectedRows, + uint64_t& resultRows) { + bool success = false; + bool shouldCancel = false; + + // This is needed to ensure the job query would be staying alive for the duration + // of the operation to prevent inconsistency within the application. + auto const uberJob = getUberJob().lock(); + if (uberJob == nullptr) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, uberJob was NULL"); + return {success, shouldCancel}; // both should still be false + } + + LOGS(_log, LOG_LVL_TRACE, + "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); + + success = _mergeHttp(uberJob, fileUrl, fileSize); + // &&& FOULED_RESULTS need to do something about shouldCancel. + // &&& until there is some way to know if csvStream has merged any bytes, just assume it has fouled the + // results. + if (!success) shouldCancel = true; + + if (!success || shouldCancel) { + LOGS(_log, LOG_LVL_WARN, __func__ << " success=" << success << " shouldCancel=" << shouldCancel); + } + + if (success) { + _infileMerger->mergeCompleteFor(uberJob->getUjId()); + } + return {success, shouldCancel}; +} + +/* &&& bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, proto::ResponseData const& responseData) { if (_flushed) { @@ -404,6 +517,7 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe return {success, shouldCancel}; } +>>>>>>> a27525c04017db9a30061fa0bb4b5228c0c5d1b2 */ void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int errState) { if (!_errorSet.exchange(true)) { _setError(errorCode, errorMsg, errState); diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 7e33fe9cfd..252ecc74bd 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -70,8 +70,12 @@ class MergingHandler : public qdisp::ResponseHandler { /// @see ResponseHandler::flushHttp /// @see MerginHandler::_mergeHttp - std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, - uint64_t& resultRows) override; + std::tuple flushHttp( + std::string const& fileUrl, uint64_t fileSize, uint64_t expectedRows, + /* &&& + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + >>>>>>> a27525c04017db9a30061fa0bb4b5228c0c5d1b2 */ + uint64_t& resultRows) override; /// @see ResponseHandler::flushHttpError void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; @@ -84,7 +88,9 @@ class MergingHandler : public qdisp::ResponseHandler { private: /// Call InfileMerger to do the work of merging this data to the result. - bool _mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + + bool _mergeHttp(std::shared_ptr const& uberJob, std::string const& fileUrl, + uint64_t fileSize); /// Set error code and string. void _setError(int code, std::string const& msg, int errorState); diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 71843811e4..61ac3f10f4 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -135,7 +135,7 @@ std::shared_ptr _makeUserQueryProcessList(query::SelectStmt::Ptr& stm LOGS(_log, LOG_LVL_DEBUG, "SELECT query is a PROCESSLIST"); try { return std::make_shared(stmt, sharedResources->qMetaSelect, - sharedResources->czarId, userQueryId, resultDb); + sharedResources->qMetaCzarId, userQueryId, resultDb); } catch (std::exception const& exc) { return std::make_shared(exc.what()); } diff --git a/src/ccontrol/UserQueryProcessList.cc b/src/ccontrol/UserQueryProcessList.cc index 88c37c5ea0..83ff05eb7a 100644 --- a/src/ccontrol/UserQueryProcessList.cc +++ b/src/ccontrol/UserQueryProcessList.cc @@ -35,6 +35,7 @@ // Qserv headers #include "css/CssAccess.h" #include "css/CssError.h" +#include "cconfig/CzarConfig.h" #include "qmeta/MessageStore.h" #include "qmeta/Exceptions.h" #include "qmeta/QMetaSelect.h" @@ -65,8 +66,7 @@ UserQueryProcessList::UserQueryProcessList(std::shared_ptr co std::shared_ptr const& qMetaSelect, qmeta::CzarId czarId, std::string const& userQueryId, std::string const& resultDb) - : _resultDbConn(resultDbConn), - _qMetaSelect(qMetaSelect), + : _qMetaSelect(qMetaSelect), _qMetaCzarId(qMetaCzarId), _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), @@ -93,8 +93,7 @@ UserQueryProcessList::UserQueryProcessList(std::shared_ptr co UserQueryProcessList::UserQueryProcessList(bool full, std::shared_ptr const& qMetaSelect, qmeta::CzarId czarId, std::string const& userQueryId, std::string const& resultDb) - : _resultDbConn(resultDbConn), - _qMetaSelect(qMetaSelect), + : _qMetaSelect(qMetaSelect), _qMetaCzarId(qMetaCzarId), _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), diff --git a/src/ccontrol/UserQueryResources.cc b/src/ccontrol/UserQueryResources.cc index aced1baeb7..1becf418bc 100644 --- a/src/ccontrol/UserQueryResources.cc +++ b/src/ccontrol/UserQueryResources.cc @@ -28,6 +28,7 @@ #include "cconfig/CzarConfig.h" #include "qmeta/QMeta.h" #include "qmeta/QProgress.h" + namespace lsst::qserv::ccontrol { UserQuerySharedResources::UserQuerySharedResources( diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 8ba8e512ad..a1b8faf6ef 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -582,11 +582,10 @@ QueryState UserQuerySelect::join() { /// Release resources held by the merger void UserQuerySelect::_discardMerger(std::lock_guard const& lock) { - _infileMergerConfig.reset(); if (_infileMerger && !_infileMerger->isFinished()) { throw UserQueryError(getQueryIdString() + " merger unfinished, cannot discard"); } - _infileMerger.reset(); + _infileMergerConfig.reset(); } /// Release resources. @@ -608,9 +607,8 @@ void UserQuerySelect::discard() { throw UserQueryError(getQueryIdString() + " Executive unfinished, cannot discard"); } + // Deleting the executive may save some time if results were found early. _executive.reset(); - _messageStore.reset(); - _qSession.reset(); try { _discardMerger(lock); diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index f009dbbae7..d0f23ef666 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -29,7 +29,8 @@ // Qserv headers #include "cconfig/CzarConfig.h" #include "czar/Czar.h" -#include "protojson/JobReadyMsg.h" +#include "protojson/UberJobErrorMsg.h" +#include "protojson/UberJobReadyMsg.h" #include "protojson/WorkerCzarComIssue.h" #include "qdisp/Executive.h" #include "qdisp/UberJob.h" @@ -107,14 +108,14 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { // Parse and verify the json message and then kill the UberJob. json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - // TODO:UJ see wbase::UberJobData::responseError for message construction - string const targetWorkerId = body().required("workerid"); - string const czarName = body().required("czar"); - qmeta::CzarId const czarId = body().required("czarid"); - QueryId const queryId = body().required("queryid"); - UberJobId const uberJobId = body().required("uberjobid"); - int const errorCode = body().required("errorCode"); - string const errorMsg = body().required("errorMsg"); + string const& repliInstanceId = cconfig::CzarConfig::instance()->replicationInstanceId(); + string const& repliAuthKey = cconfig::CzarConfig::instance()->replicationAuthKey(); + auto const& jsReq = body().objJson; + auto jrMsg = protojson::UberJobErrorMsg::createFromJson(jsReq, repliInstanceId, repliAuthKey); + + auto const queryId = jrMsg->getQueryId(); + auto const czarId = jrMsg->getCzarId(); + auto const uberJobId = jrMsg->getUberJobId(); // Find UberJob qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); @@ -129,9 +130,8 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { " czar=" + to_string(czarId)); } - auto importRes = uj->workerError(errorCode, errorMsg); + auto importRes = uj->workerError(jrMsg->getErrorCode(), jrMsg->getErrorMsg()); jsRet = importRes; - } catch (std::invalid_argument const& iaEx) { LOGS(_log, LOG_LVL_ERROR, "HttpCzarWorkerModule::_handleJobError received " << iaEx.what() << " js=" << body().objJson); @@ -148,10 +148,8 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { // Parse and verify the json message and then have the uberjob import the file. json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { - string const repliInstanceId = cconfig::CzarConfig::instance()->replicationInstanceId(); - string const repliAuthKey = cconfig::CzarConfig::instance()->replicationAuthKey(); auto const& jsReq = body().objJson; - auto jrMsg = protojson::JobReadyMsg::createFromJson(jsReq, repliInstanceId, repliAuthKey); + auto jrMsg = protojson::UberJobReadyMsg::createFromJson(jsReq); // Find UberJob auto queryId = jrMsg->getQueryId(); @@ -212,7 +210,7 @@ json HttpCzarWorkerModule::_handleWorkerCzarComIssue(string const& func) { execPtr->killIncompleteUberJobsOnWorker(wId); } } - jsRet = wccIssue->serializeResponseJson(); + jsRet = wccIssue->responseToJson(); LOGS(_log, LOG_LVL_TRACE, "HttpCzarWorkerModule::_handleWorkerCzarComIssue jsRet=" << jsRet.dump()); } catch (std::invalid_argument const& iaEx) { diff --git a/src/mysql/CMakeLists.txt b/src/mysql/CMakeLists.txt index 3438141219..fd5435b75a 100644 --- a/src/mysql/CMakeLists.txt +++ b/src/mysql/CMakeLists.txt @@ -2,11 +2,11 @@ add_library(mysql SHARED) add_dependencies(mysql proto) target_sources(mysql PRIVATE + CsvBuffer.cc LocalInfile.cc MySqlConfig.cc MySqlConnection.cc MySqlUtils.cc - CsvBuffer.cc SchemaFactory.cc ) diff --git a/src/mysql/LocalInfile.cc b/src/mysql/LocalInfile.cc index dc3a26484d..66c06a2f16 100644 --- a/src/mysql/LocalInfile.cc +++ b/src/mysql/LocalInfile.cc @@ -132,8 +132,6 @@ int LocalInfile::getError(char* buf, unsigned int bufLen) { return 0; } -LocalInfile::Mgr::~Mgr() { LOGS(_log, LOG_LVL_TRACE, "LocalInfile::Mgr::~Mgr()"); } - void LocalInfile::Mgr::attach(MYSQL* mysql) { mysql_set_local_infile_handler(mysql, local_infile_init, local_infile_read, local_infile_end, local_infile_error, this); @@ -156,7 +154,11 @@ std::string LocalInfile::Mgr::prepareSrc(std::shared_ptr const& csvBu int LocalInfile::Mgr::local_infile_init(void** ptr, const char* filename, void* userdata) { assert(userdata); LocalInfile::Mgr* m = static_cast(userdata); +<<<<<<< HEAD auto csvBuffer = m->get(std::string(filename)); +======= + auto csvBuffer = m->getCsv(std::string(filename)); +>>>>>>> 24f5d37ea (Added JobErrorMsg.) assert(csvBuffer); LocalInfile* lf = new LocalInfile(filename, csvBuffer); *ptr = lf; @@ -190,10 +192,17 @@ void LocalInfile::Mgr::setBuffer(std::string const& filename, std::shared_ptr LocalInfile::Mgr::get(std::string const& filename) { std::lock_guard lock(_mapMutex); CsvBufferMap::iterator i = _map.find(filename); if (i == _map.end()) { +======= +std::shared_ptr LocalInfile::Mgr::getCsv(std::string const& filename) { + std::lock_guard lock(_mapMutex); + auto i = _mapCsv.find(filename); + if (i == _mapCsv.end()) { +>>>>>>> 24f5d37ea (Added JobErrorMsg.) return std::shared_ptr(); } return i->second; @@ -209,7 +218,11 @@ std::string LocalInfile::Mgr::_nextFilename() { bool LocalInfile::Mgr::_set(std::string const& filename, std::shared_ptr const& csvBuffer) { std::lock_guard lock(_mapMutex); +<<<<<<< HEAD auto res = _map.insert(std::pair>(filename, csvBuffer)); +======= + auto res = _mapCsv.insert(std::pair>(filename, csvBuffer)); +>>>>>>> 24f5d37ea (Added JobErrorMsg.) return res.second; } diff --git a/src/mysql/LocalInfile.h b/src/mysql/LocalInfile.h index d9c46ba93b..9c03b16b34 100644 --- a/src/mysql/LocalInfile.h +++ b/src/mysql/LocalInfile.h @@ -125,7 +125,7 @@ class LocalInfile::Mgr : boost::noncopyable { std::string insertBuffer(std::shared_ptr const& csvBuffer); void setBuffer(std::string const& s, std::shared_ptr const& csvBuffer); - std::shared_ptr get(std::string const& filename); + std::shared_ptr getCsv(std::string const& filename); private: /// @return next filename @@ -135,7 +135,7 @@ class LocalInfile::Mgr : boost::noncopyable { bool _set(std::string const& filename, std::shared_ptr const& csvBuffer); typedef std::map> CsvBufferMap; - CsvBufferMap _map; + CsvBufferMap _mapCsv; std::mutex _mapMutex; }; diff --git a/src/protojson/CMakeLists.txt b/src/protojson/CMakeLists.txt index 35e74c708d..08bbdff09d 100644 --- a/src/protojson/CMakeLists.txt +++ b/src/protojson/CMakeLists.txt @@ -1,8 +1,9 @@ add_library(protojson SHARED) target_sources(protojson PRIVATE - JobReadyMsg.cc - ScanTableInfo.cc + ScanTableInfo.cc + UberJobErrorMsg.cc + UberJobReadyMsg.cc UberJobMsg.cc WorkerCzarComIssue.cc WorkerQueryStatusData.cc @@ -37,7 +38,8 @@ function(PROTOJSON_TESTS) endfunction() protojson_tests( - testJobReadyMsg testStatusData + testUberJobErrorMsg + testUberJobReadyMsg testUberJobMsg ) diff --git a/src/protojson/JobReadyMsg.cc b/src/protojson/JobReadyMsg.cc deleted file mode 100644 index cccd230cbb..0000000000 --- a/src/protojson/JobReadyMsg.cc +++ /dev/null @@ -1,116 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "protojson/JobReadyMsg.h" - -#include - -// Qserv headers -#include "http/Client.h" -#include "http/MetaModule.h" -#include "http/RequestBodyJSON.h" -#include "util/common.h" -#include "util/TimeUtils.h" - -// LSST headers -#include "lsst/log/Log.h" - -using namespace std; -using namespace nlohmann; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.JobReadyMsg"); -} // namespace - -namespace lsst::qserv::protojson { - -JobReadyMsg::Ptr JobReadyMsg::create(std::string const& replicationInstanceId, - std::string const& replicationAuthKey, std::string const& workerIdStr, - std::string const& czarName, CzarIdType czarId, QueryId queryId, - UberJobId uberJobId, std::string const& fileUrl, uint64_t rowCount, - uint64_t fileSize) { - auto jrMsg = Ptr(new JobReadyMsg(replicationInstanceId, replicationAuthKey)); - jrMsg->_workerId = workerIdStr; - jrMsg->_czarName = czarName; - jrMsg->_czarId = czarId; - jrMsg->_queryId = queryId; - jrMsg->_uberJobId = uberJobId; - jrMsg->_fileUrl = fileUrl; - jrMsg->_rowCount = rowCount; - jrMsg->_fileSize = fileSize; - return jrMsg; -} - -JobReadyMsg::Ptr JobReadyMsg::createFromJson(nlohmann::json const& jsWReq, - std::string const& replicationInstanceId, - std::string const& replicationAuthKey) { - string const fName("JobReadyMsg::createFromJson"); - LOGS(_log, LOG_LVL_DEBUG, fName); - try { - if (jsWReq["version"] != http::MetaModule::version) { - LOGS(_log, LOG_LVL_ERROR, fName << " bad version"); - return nullptr; - } - - // Presumably, if these were wrong, it wouldn't have gotten this far. - auto repliInstId = http::RequestBodyJSON::required(jsWReq, "instance_id"); - auto repliAuthKey = http::RequestBodyJSON::required(jsWReq, "auth_key"); - - auto jrMsg = create(repliInstId, repliAuthKey); - - jrMsg->_workerId = http::RequestBodyJSON::required(jsWReq, "workerid"); - jrMsg->_czarName = http::RequestBodyJSON::required(jsWReq, "czar"); - jrMsg->_czarId = http::RequestBodyJSON::required(jsWReq, "czarid"); - jrMsg->_queryId = http::RequestBodyJSON::required(jsWReq, "queryid"); - jrMsg->_uberJobId = http::RequestBodyJSON::required(jsWReq, "uberjobid"); - jrMsg->_fileUrl = http::RequestBodyJSON::required(jsWReq, "fileUrl"); - jrMsg->_rowCount = http::RequestBodyJSON::required(jsWReq, "rowCount"); - jrMsg->_fileSize = http::RequestBodyJSON::required(jsWReq, "fileSize"); - return jrMsg; - } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_ERROR, string("JobReadyMsg::createJson invalid ") << exc.what()); - } - return nullptr; -} - -json JobReadyMsg::serializeJson() { - shared_ptr jsJrReqPtr = make_shared(); - json& jsJr = *jsJrReqPtr; - - // These need to match what http::BaseModule::enforceInstanceId() - // and http::BaseModule::enforceAuthorization() are looking for. - jsJr["instance_id"] = _replicationInstanceId; - jsJr["auth_key"] = _replicationAuthKey; - - jsJr["version"] = http::MetaModule::version; - jsJr["workerid"] = _workerId; - jsJr["czar"] = _czarName; - jsJr["czarid"] = _czarId; - jsJr["queryid"] = _queryId; - jsJr["uberjobid"] = _uberJobId; - jsJr["fileUrl"] = _fileUrl; - jsJr["rowCount"] = _rowCount; - jsJr["fileSize"] = _fileSize; - return jsJr; -} - -} // namespace lsst::qserv::protojson diff --git a/src/protojson/UberJobErrorMsg.cc b/src/protojson/UberJobErrorMsg.cc new file mode 100644 index 0000000000..ff23c20033 --- /dev/null +++ b/src/protojson/UberJobErrorMsg.cc @@ -0,0 +1,124 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "protojson/UberJobErrorMsg.h" + +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "util/common.h" +#include "util/TimeUtils.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.UberJobErrorMsg"); +} // namespace + +namespace lsst::qserv::protojson { + +string UberJobErrorMsg::_cName(const char* fName) const { + return string("UberJobErrorMsg::") + fName + " qId=" + to_string(_queryId) + + " ujId=" + to_string(_uberJobId); +} + +UberJobErrorMsg::Ptr UberJobErrorMsg::create(string const& replicationInstanceId, + string const& replicationAuthKey, unsigned int version, + string const& workerIdStr, string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, + int errorCode, string const& errorMsg) { + Ptr jrMsg = Ptr(new UberJobErrorMsg(replicationInstanceId, replicationAuthKey, version, workerIdStr, + czarName, czarId, queryId, uberJobId, errorCode, errorMsg)); + return jrMsg; +} + +UberJobErrorMsg::Ptr UberJobErrorMsg::createFromJson(nlohmann::json const& jsWReq, + string const& replicationInstanceId, + string const& replicationAuthKey) { + string const fName("UberJobErrorMsg::createFromJson"); + LOGS(_log, LOG_LVL_DEBUG, fName); + try { + Ptr jrMsg = Ptr(new UberJobErrorMsg(http::RequestBodyJSON::required(jsWReq, "instance_id"), + http::RequestBodyJSON::required(jsWReq, "auth_key"), + http::RequestBodyJSON::required(jsWReq, "version"), + http::RequestBodyJSON::required(jsWReq, "workerid"), + http::RequestBodyJSON::required(jsWReq, "czar"), + http::RequestBodyJSON::required(jsWReq, "czarid"), + http::RequestBodyJSON::required(jsWReq, "queryid"), + http::RequestBodyJSON::required(jsWReq, "uberjobid"), + http::RequestBodyJSON::required(jsWReq, "errorCode"), + http::RequestBodyJSON::required(jsWReq, "errorMsg"))); + return jrMsg; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("UberJobErrorMsg::createJson invalid ") << exc.what()); + } + return nullptr; +} + +UberJobErrorMsg::UberJobErrorMsg(string const& replicationInstanceId, string const& replicationAuthKey, + unsigned int version, string const& workerId, string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, int errorCode, + string const& errorMsg) + : _replicationInstanceId(replicationInstanceId), + _replicationAuthKey(replicationAuthKey), + _version(version), + _workerId(workerId), + _czarName(czarName), + _czarId(czarId), + _queryId(queryId), + _uberJobId(uberJobId), + _errorCode(errorCode), + _errorMsg(errorMsg) { + if (_version != http::MetaModule::version) { + string eMsg = _cName(__func__) + " bad version " + to_string(_version); + LOGS(_log, LOG_LVL_ERROR, eMsg); + throw invalid_argument(eMsg); + } +} + +json UberJobErrorMsg::toJson() const { + json jsJr; + + // These need to match what http::BaseModule::enforceInstanceId() + // and http::BaseModule::enforceAuthorization() are looking for. + jsJr["instance_id"] = _replicationInstanceId; + jsJr["auth_key"] = _replicationAuthKey; + jsJr["version"] = _version; + + jsJr["workerid"] = _workerId; + jsJr["czar"] = _czarName; + jsJr["czarid"] = _czarId; + jsJr["queryid"] = _queryId; + jsJr["uberjobid"] = _uberJobId; + jsJr["errorCode"] = _errorCode; + jsJr["errorMsg"] = _errorMsg; + return jsJr; +} + +} // namespace lsst::qserv::protojson diff --git a/src/protojson/JobReadyMsg.h b/src/protojson/UberJobErrorMsg.h similarity index 52% rename from src/protojson/JobReadyMsg.h rename to src/protojson/UberJobErrorMsg.h index 4105421256..ce2e8c6e7e 100644 --- a/src/protojson/JobReadyMsg.h +++ b/src/protojson/UberJobErrorMsg.h @@ -18,8 +18,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_PROTOJSON_JOBREADYMSG_H -#define LSST_QSERV_PROTOJSON_JOBREADYMSG_H +#ifndef LSST_QSERV_PROTOJSON_UBERJOBERRORMSG_H +#define LSST_QSERV_PROTOJSON_UBERJOBERRORMSG_H // System headers #include @@ -39,61 +39,60 @@ // This header declarations namespace lsst::qserv::protojson { -/// This class handles the message used to inform the czar that a result file -/// for an UberJob is ready. -class JobReadyMsg { +/// This class handles the message used to inform the czar that there has +/// been a problem with an UberJob. +class UberJobErrorMsg { public: - using Ptr = std::shared_ptr; + using Ptr = std::shared_ptr; - JobReadyMsg() = delete; - JobReadyMsg(std::string const& replicationInstanceId, std::string const& replicationAuthKey) - : _replicationInstanceId(replicationInstanceId), _replicationAuthKey(replicationAuthKey) {} - JobReadyMsg(JobReadyMsg const&) = delete; - JobReadyMsg& operator=(JobReadyMsg const&) = delete; + UberJobErrorMsg(std::string const& replicationInstanceId, std::string const& replicationAuthKey, + unsigned int version, std::string const& workerId, std::string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, int errorCode, + std::string const& errorMsg); - std::string cName(const char* fName) { return std::string("WorkerQueryStatusData::") + fName; } + UberJobErrorMsg() = delete; + UberJobErrorMsg(UberJobErrorMsg const&) = delete; + UberJobErrorMsg& operator=(UberJobErrorMsg const&) = delete; static Ptr create(std::string const& replicationInstanceId, std::string const& replicationAuthKey, - std::string const& workerIdStr, std::string const& czarName, CzarIdType czarId, - QueryId queryId, UberJobId uberJobId, std::string const& fileUrl, uint64_t rowCount, - uint64_t fileSize); + unsigned int version, std::string const& workerIdStr, std::string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, int errorCode, + std::string const& errorMsg); - static Ptr create(std::string const& replicationInstanceId, std::string const& replicationAuthKey) { - return Ptr(new JobReadyMsg(replicationInstanceId, replicationAuthKey)); - } - - /// This function creates a JobReadyMsg object from the worker json `czarJson`, the + /// This function creates a UberJobErrorMsg object from the worker json `czarJson`, the /// other parameters are used to verify the json message. static Ptr createFromJson(nlohmann::json const& czarJson, std::string const& replicationInstanceId, std::string const& replicationAuthKey); - ~JobReadyMsg() = default; + ~UberJobErrorMsg() = default; /// Return a json object with data allowing collection of UberJob result file. - nlohmann::json serializeJson(); + nlohmann::json toJson() const; - std::string getWorkerId() const { return _workerId; } - std::string getCzarName() const { return _czarName; } + std::string const& getWorkerId() const { return _workerId; } + std::string const& getCzarName() const { return _czarName; } CzarIdType getCzarId() const { return _czarId; } QueryId getQueryId() const { return _queryId; } UberJobId getUberJobId() const { return _uberJobId; } - std::string getFileUrl() const { return _fileUrl; } - uint64_t getRowCount() const { return _rowCount; } - uint64_t getFileSize() const { return _fileSize; } + std::string const& getErrorMsg() const { return _errorMsg; } + uint getErrorCode() const { return _errorCode; } private: + /// class name for log, fName is expected to be __func__. + std::string _cName(const char* fName) const; + std::string const _replicationInstanceId; std::string const _replicationAuthKey; - std::string _workerId; - std::string _czarName; - CzarIdType _czarId = 0; - QueryId _queryId = 0; - UberJobId _uberJobId = 0; - std::string _fileUrl; - uint64_t _rowCount = 0; - uint64_t _fileSize = 0; + unsigned int const _version; + std::string const _workerId; + std::string const _czarName; + CzarIdType const _czarId; + QueryId const _queryId; + UberJobId const _uberJobId; + int const _errorCode; + std::string const _errorMsg; }; } // namespace lsst::qserv::protojson -#endif // LSST_QSERV_PROTOJSON_JOBREADYMSG_H +#endif // LSST_QSERV_PROTOJSON_UBERJOBERRORMSG_H diff --git a/src/protojson/UberJobReadyMsg.cc b/src/protojson/UberJobReadyMsg.cc new file mode 100644 index 0000000000..43ea274dca --- /dev/null +++ b/src/protojson/UberJobReadyMsg.cc @@ -0,0 +1,127 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "protojson/UberJobReadyMsg.h" + +#include + +// Qserv headers +#include "http/Client.h" +#include "http/MetaModule.h" +#include "http/RequestBodyJSON.h" +#include "util/common.h" +#include "util/TimeUtils.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; +using namespace nlohmann; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.UberJobReadyMsg"); +} // namespace + +namespace lsst::qserv::protojson { + +string UberJobReadyMsg::_cName(const char* fName) const { + return string("UberJobReadyMsg::") + fName + " qId=" + to_string(_queryId) + + " ujId=" + to_string(_uberJobId); +} + +UberJobReadyMsg::Ptr UberJobReadyMsg::create(string const& replicationInstanceId, + string const& replicationAuthKey, unsigned int version, + string const& workerIdStr, string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, + string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { + Ptr jrMsg = Ptr(new UberJobReadyMsg(replicationInstanceId, replicationAuthKey, version, workerIdStr, + czarName, czarId, queryId, uberJobId, fileUrl, rowCount, fileSize)); + return jrMsg; +} + +UberJobReadyMsg::Ptr UberJobReadyMsg::createFromJson(json const& jsWReq) { + string const fName("UberJobReadyMsg::createFromJson"); + LOGS(_log, LOG_LVL_DEBUG, fName); + try { + // If replication identifiers were wrong, it wouldn't have gotten this far. + Ptr jrMsg = Ptr(new UberJobReadyMsg(http::RequestBodyJSON::required(jsWReq, "instance_id"), + http::RequestBodyJSON::required(jsWReq, "auth_key"), + http::RequestBodyJSON::required(jsWReq, "version"), + http::RequestBodyJSON::required(jsWReq, "workerid"), + http::RequestBodyJSON::required(jsWReq, "czar"), + http::RequestBodyJSON::required(jsWReq, "czarid"), + http::RequestBodyJSON::required(jsWReq, "queryid"), + http::RequestBodyJSON::required(jsWReq, "uberjobid"), + http::RequestBodyJSON::required(jsWReq, "fileUrl"), + http::RequestBodyJSON::required(jsWReq, "rowCount"), + http::RequestBodyJSON::required(jsWReq, "fileSize"))); + return jrMsg; + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_ERROR, string("UberJobReadyMsg::createJson invalid ") << exc.what()); + } + return nullptr; +} + +UberJobReadyMsg::UberJobReadyMsg(string const& replicationInstanceId, string const& replicationAuthKey, + unsigned int version, string const& workerId, string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, + string const& fileUrl, uint64_t rowCount, uint64_t fileSize) + : _replicationInstanceId(replicationInstanceId), + _replicationAuthKey(replicationAuthKey), + _version(version), + _workerId(workerId), + _czarName(czarName), + _czarId(czarId), + _queryId(queryId), + _uberJobId(uberJobId), + _fileUrl(fileUrl), + _rowCount(rowCount), + _fileSize(fileSize) { + if (_version != http::MetaModule::version) { + string eMsg = _cName(__func__) + " bad version " + to_string(_version); + LOGS(_log, LOG_LVL_ERROR, eMsg); + throw invalid_argument(eMsg); + } +} + +json UberJobReadyMsg::toJson() const { + shared_ptr jsJrReqPtr = make_shared(); + json& jsJr = *jsJrReqPtr; + + // These need to match what http::BaseModule::enforceInstanceId() + // and http::BaseModule::enforceAuthorization() are looking for. + jsJr["instance_id"] = _replicationInstanceId; + jsJr["auth_key"] = _replicationAuthKey; + jsJr["version"] = _version; + + jsJr["workerid"] = _workerId; + jsJr["czar"] = _czarName; + jsJr["czarid"] = _czarId; + jsJr["queryid"] = _queryId; + jsJr["uberjobid"] = _uberJobId; + jsJr["fileUrl"] = _fileUrl; + jsJr["rowCount"] = _rowCount; + jsJr["fileSize"] = _fileSize; + return jsJr; +} + +} // namespace lsst::qserv::protojson diff --git a/src/protojson/UberJobReadyMsg.h b/src/protojson/UberJobReadyMsg.h new file mode 100644 index 0000000000..524a4d81f3 --- /dev/null +++ b/src/protojson/UberJobReadyMsg.h @@ -0,0 +1,99 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_PROTOJSON_UBERJOBREADYMSG_H +#define LSST_QSERV_PROTOJSON_UBERJOBREADYMSG_H + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "nlohmann/json.hpp" + +// qserv headers +#include "global/clock_defs.h" +#include "global/intTypes.h" +#include "protojson/WorkerQueryStatusData.h" + +// This header declarations +namespace lsst::qserv::protojson { + +/// This class handles the message used to inform the czar that a result file +/// for an UberJob is ready. +class UberJobReadyMsg { +public: + using Ptr = std::shared_ptr; + + UberJobReadyMsg(std::string const& replicationInstanceId, std::string const& replicationAuthKey, + unsigned int version, std::string const& workerId, std::string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, std::string const& fileUrl, + uint64_t rowCount, uint64_t fileSize); + + UberJobReadyMsg() = delete; + UberJobReadyMsg(UberJobReadyMsg const&) = delete; + UberJobReadyMsg& operator=(UberJobReadyMsg const&) = delete; + + static Ptr create(std::string const& replicationInstanceId, std::string const& replicationAuthKey, + unsigned int version, std::string const& workerIdStr, std::string const& czarName, + CzarIdType czarId, QueryId queryId, UberJobId uberJobId, std::string const& fileUrl, + uint64_t rowCount, uint64_t fileSize); + + /// This function creates a UberJobReadyMsg object from the worker json `czarJson`, the + /// other parameters are used to verify the json message. + static Ptr createFromJson(nlohmann::json const& czarJson); + + ~UberJobReadyMsg() = default; + + /// Return a json object with data allowing collection of UberJob result file. + nlohmann::json toJson() const; + + std::string const& getWorkerId() const { return _workerId; } + std::string const& getCzarName() const { return _czarName; } + CzarIdType getCzarId() const { return _czarId; } + QueryId getQueryId() const { return _queryId; } + UberJobId getUberJobId() const { return _uberJobId; } + std::string const& getFileUrl() const { return _fileUrl; } + uint64_t getRowCount() const { return _rowCount; } + uint64_t getFileSize() const { return _fileSize; } + +private: + /// class name for log, fName is expected to be __func__. + std::string _cName(const char* fName) const; + + std::string const _replicationInstanceId; + std::string const _replicationAuthKey; + unsigned int const _version; + std::string const _workerId; + std::string const _czarName; + CzarIdType const _czarId; + QueryId const _queryId; + UberJobId const _uberJobId; + std::string const _fileUrl; + uint64_t const _rowCount; + uint64_t const _fileSize; +}; + +} // namespace lsst::qserv::protojson + +#endif // LSST_QSERV_PROTOJSON_UBERJOBREADYMSG_H diff --git a/src/protojson/WorkerCzarComIssue.cc b/src/protojson/WorkerCzarComIssue.cc index 1a9a10bbd9..c55b20b0c3 100644 --- a/src/protojson/WorkerCzarComIssue.cc +++ b/src/protojson/WorkerCzarComIssue.cc @@ -43,7 +43,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.WorkerCzarComIssue"); namespace lsst::qserv::protojson { -shared_ptr WorkerCzarComIssue::serializeJson() { +shared_ptr WorkerCzarComIssue::toJson() const { shared_ptr jsCzarReqPtr = make_shared(); json& jsCzarR = *jsCzarReqPtr; lock_guard _lgWciMtx(_wciMtx); @@ -94,7 +94,7 @@ WorkerCzarComIssue::Ptr WorkerCzarComIssue::createFromJson(nlohmann::json const& return nullptr; } -json WorkerCzarComIssue::serializeResponseJson() { +json WorkerCzarComIssue::responseToJson() const { json jsResp = {{"success", 1}, {"errortype", "none"}, {"note", ""}}; // TODO:UJ add lists of uberjobs that are scheduled to have files collected because of this message. diff --git a/src/protojson/WorkerCzarComIssue.h b/src/protojson/WorkerCzarComIssue.h index 21a0e53cdb..505c3509bf 100644 --- a/src/protojson/WorkerCzarComIssue.h +++ b/src/protojson/WorkerCzarComIssue.h @@ -59,7 +59,7 @@ class WorkerCzarComIssue { WorkerCzarComIssue() = delete; ~WorkerCzarComIssue() = default; - std::string cName(const char* funcN) { return std::string("WorkerCzarComIssue") + funcN; } + std::string cName(const char* funcN) const { return std::string("WorkerCzarComIssue") + funcN; } static Ptr create(std::string const& replicationInstanceId_, std::string const& replicationAuthKey_) { return Ptr(new WorkerCzarComIssue(replicationInstanceId_, replicationAuthKey_)); @@ -100,11 +100,11 @@ class WorkerCzarComIssue { } /// Return a json version of the contents of this class. - std::shared_ptr serializeJson(); + std::shared_ptr toJson() const; /// Return a json object indicating the status of the message for the /// original requester. - nlohmann::json serializeResponseJson(); + nlohmann::json responseToJson() const; std::string dump() const; diff --git a/src/protojson/testStatusData.cc b/src/protojson/testStatusData.cc index 9b63827ea7..3135338568 100644 --- a/src/protojson/testStatusData.cc +++ b/src/protojson/testStatusData.cc @@ -169,11 +169,11 @@ BOOST_AUTO_TEST_CASE(WorkerCzarComIssue) { wccIssueA->setThoughtCzarWasDead(true); BOOST_REQUIRE(wccIssueA->needToSend() == true); - auto jsIssueA = wccIssueA->serializeJson(); + auto jsIssueA = wccIssueA->toJson(); auto wccIssueA1 = lsst::qserv::protojson::WorkerCzarComIssue::createFromJson( *jsIssueA, replicationInstanceId, replicationAuthKey); - auto jsIssueA1 = wccIssueA1->serializeJson(); + auto jsIssueA1 = wccIssueA1->toJson(); BOOST_REQUIRE(*jsIssueA == *jsIssueA1); // TODO:UJ Test with items in lists. diff --git a/src/protojson/testUberJobErrorMsg.cc b/src/protojson/testUberJobErrorMsg.cc new file mode 100644 index 0000000000..df9f2af41e --- /dev/null +++ b/src/protojson/testUberJobErrorMsg.cc @@ -0,0 +1,106 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// System headers +#include +#include +#include +#include +#include + +#include "nlohmann/json.hpp" + +// Qserv headers +#include "global/clock_defs.h" +#include "http/MetaModule.h" +#include "lsst/log/Log.h" +#include "protojson/UberJobErrorMsg.h" + +// Boost unit test header +#define BOOST_TEST_MODULE RequestQuery +#include + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.testUberJobErrorMsg"); +} + +using namespace std; +namespace test = boost::test_tools; +using namespace lsst::qserv::protojson; + +string const repliInstanceId = "repliInstId"; +string const repliAuthKey = "repliIAuthKey"; +unsigned int const version = lsst::qserv::http::MetaModule::version; + +BOOST_AUTO_TEST_SUITE(Suite) + +bool parseSerializeReparseCheck(string const& jsStr, string const& note) { + string fName("parseSerialize "); + fName += note + " "; + LOGS(_log, LOG_LVL_INFO, fName << " start " << jsStr); + nlohmann::json js = nlohmann::json::parse(jsStr); + LOGS(_log, LOG_LVL_INFO, fName << " parse 1"); + + UberJobErrorMsg::Ptr jrm = UberJobErrorMsg::createFromJson(js, repliInstanceId, repliAuthKey); + BOOST_REQUIRE(jrm != nullptr); + + nlohmann::json jsJrm = jrm->toJson(); + LOGS(_log, LOG_LVL_INFO, fName << " serialized jsJrm=" << jsJrm); + + UberJobErrorMsg::Ptr jrmCreated = UberJobErrorMsg::createFromJson(jsJrm, repliInstanceId, repliAuthKey); + LOGS(_log, LOG_LVL_INFO, fName << " created"); + nlohmann::json jsJrmCreated = jrmCreated->toJson(); + LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); + + bool createdMatchesOriginal = jsJrm == jsJrmCreated; + if (createdMatchesOriginal) { + LOGS(_log, LOG_LVL_INFO, fName << "created matches original"); + } else { + LOGS(_log, LOG_LVL_ERROR, "jsJrm != jsJrmCreated"); + LOGS(_log, LOG_LVL_ERROR, "jsJrm=" << jsJrm); + LOGS(_log, LOG_LVL_ERROR, "jsJrmCreated=" << jsJrmCreated); + } + BOOST_REQUIRE(createdMatchesOriginal); + return createdMatchesOriginal; +} + +BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { + LOGS(_log, LOG_LVL_INFO, "testJRM start"); + + string const workerIdStr("wrker72"); + string const czarName("cz4242"); + lsst::qserv::CzarIdType const czarId = 745; + lsst::qserv::QueryId const queryId = 986532; + lsst::qserv::UberJobId const uberJobId = 14578; + string const errorMsg("something went wrong"); + int const errorCode = -3; + + auto jrm = UberJobErrorMsg::create(repliInstanceId, repliAuthKey, version, workerIdStr, czarName, czarId, + queryId, uberJobId, errorCode, errorMsg); + + auto jsJrm = jrm->toJson(); + string const strJrm = to_string(jsJrm); + LOGS(_log, LOG_LVL_INFO, "stdJrm=" << strJrm); + + BOOST_REQUIRE(parseSerializeReparseCheck(strJrm, "A")); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/src/protojson/testJobReadyMsg.cc b/src/protojson/testUberJobReadyMsg.cc similarity index 82% rename from src/protojson/testJobReadyMsg.cc rename to src/protojson/testUberJobReadyMsg.cc index 339bd8e3d3..f7203a6155 100644 --- a/src/protojson/testJobReadyMsg.cc +++ b/src/protojson/testUberJobReadyMsg.cc @@ -32,14 +32,14 @@ #include "global/clock_defs.h" #include "http/MetaModule.h" #include "lsst/log/Log.h" -#include "protojson/JobReadyMsg.h" +#include "protojson/UberJobReadyMsg.h" // Boost unit test header #define BOOST_TEST_MODULE RequestQuery #include namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.testJobReadyMsg"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.testUberJobReadyMsg"); } using namespace std; @@ -48,6 +48,7 @@ using namespace lsst::qserv::protojson; string const repliInstanceId = "repliInstId"; string const repliAuthKey = "repliIAuthKey"; +unsigned int const version = lsst::qserv::http::MetaModule::version; BOOST_AUTO_TEST_SUITE(Suite) @@ -58,15 +59,15 @@ bool parseSerializeReparseCheck(string const& jsStr, string const& note) { nlohmann::json js = nlohmann::json::parse(jsStr); LOGS(_log, LOG_LVL_INFO, fName << " parse 1"); - JobReadyMsg::Ptr jrm = JobReadyMsg::createFromJson(js, repliInstanceId, repliAuthKey); + UberJobReadyMsg::Ptr jrm = UberJobReadyMsg::createFromJson(js); BOOST_REQUIRE(jrm != nullptr); - nlohmann::json jsJrm = jrm->serializeJson(); + nlohmann::json jsJrm = jrm->toJson(); LOGS(_log, LOG_LVL_INFO, fName << " serialized jsJrm=" << jsJrm); - JobReadyMsg::Ptr jrmCreated = JobReadyMsg::createFromJson(jsJrm, repliInstanceId, repliAuthKey); + UberJobReadyMsg::Ptr jrmCreated = UberJobReadyMsg::createFromJson(jsJrm); LOGS(_log, LOG_LVL_INFO, fName << " created"); - nlohmann::json jsJrmCreated = jrmCreated->serializeJson(); + nlohmann::json jsJrmCreated = jrmCreated->toJson(); LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); bool createdMatchesOriginal = jsJrm == jsJrmCreated; @@ -93,10 +94,10 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { uint64_t const rowCount = 391; uint64_t const fileSize = 5623; - auto jrm = JobReadyMsg::create(repliInstanceId, repliAuthKey, workerIdStr, czarName, czarId, queryId, - uberJobId, fileUrl, rowCount, fileSize); + auto jrm = UberJobReadyMsg::create(repliInstanceId, repliAuthKey, version, workerIdStr, czarName, czarId, + queryId, uberJobId, fileUrl, rowCount, fileSize); - auto jsJrm = jrm->serializeJson(); + auto jsJrm = jrm->toJson(); string const strJrm = to_string(jsJrm); LOGS(_log, LOG_LVL_INFO, "stdJrm=" << strJrm); diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 1dcf055e4f..276e680d9d 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -66,12 +66,16 @@ class ResponseHandler { virtual bool flush(proto::ResponseSummary const& responseSummary) = 0; /// Collect result data from the worker and merge it with the query result table. + /// If success, then everything is fine. + /// If not success, and not shouldCancel, the user query can be saved by abandoning + /// this UberJob. If shouldCancel is true, the result table is fouled and the user + /// query is ruined. /// @return success - true if the operation was successful /// @return shouldCancel - if success was false, this being true indicates there /// was an unrecoverable error in table writing and the query /// should be cancelled. - virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, - uint64_t& resultRows) = 0; + virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t fileSize, + uint64_t expectedRows, uint64_t& resultRows) = 0; /// Add the error to the error output if it is the first error. virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 4e7822ec37..76391ee948 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -38,6 +38,7 @@ #include "http/MetaModule.h" #include "proto/worker.pb.h" #include "protojson/UberJobMsg.h" +#include "qdisp/CzarStats.h" #include "qdisp/JobQuery.h" #include "qmeta/JobStatus.h" #include "qproc/ChunkQuerySpec.h" @@ -301,7 +302,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ // fileCollectFunc will be put on the queue to run later. string const idStr = _idStr; - auto fileCollectFunc = [ujThis, fileUrl, rowCount, idStr](util::CmdData*) { + auto fileCollectFunc = [ujThis, fileUrl, fileSize, rowCount, idStr](util::CmdData*) { auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { LOGS(_log, LOG_LVL_DEBUG, @@ -310,11 +311,14 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ } uint64_t resultRows = 0; auto [flushSuccess, mergeHappened] = - ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); + ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize, rowCount, resultRows); LOGS(_log, LOG_LVL_TRACE, ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushSuccess << " mergeHappened=" << mergeHappened); - if (!flushSuccess) { + if (flushSuccess) { + qdisp::CzarStats::get()->addTotalRowsRecv(rowCount); + qdisp::CzarStats::get()->addTotalBytesRecv(fileSize); + } else { bool flushShouldCancel = false; if (mergeHappened) { // This would probably indicate malformed file+rowCount or writing the result table failed. diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index b6bc616473..277fb6dc82 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -86,7 +86,8 @@ class TestInfo : public ResponseHandler { } // virtual function that won't be needed - std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + + std::tuple flushHttp(std::string const& fileUrl, uint64_t fileSize, uint64_t expectedRows, uint64_t& resultRows) override { return {true, false}; } diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 6176e751df..b1d8275bf2 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -57,7 +57,6 @@ #include "cconfig/CzarConfig.h" #include "global/intTypes.h" #include "mysql/CsvBuffer.h" -#include "proto/ProtoImporter.h" #include "proto/worker.pb.h" #include "qdisp/CzarStats.h" #include "qdisp/UberJob.h" @@ -165,18 +164,14 @@ void InfileMerger::mergeCompleteFor(int jobId) { _totalResultSize += _perJobResultSize[jobId]; // TODO:UJ this can probably be simplified } -bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { +bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSize, + std::shared_ptr const& csvStream) { UberJobId const uJobId = uberJob->getUjId(); std::string queryIdJobStr = uberJob->getIdStr(); if (!_queryIdStrSet) { _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); } - // Nothing to do if size is zero. - if (responseData.row_size() == 0) { - return true; - } - // Do nothing if the query got cancelled for any reason. if (uberJob->isQueryCancelled()) { return true; @@ -199,13 +194,16 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response // Add columns to rows in virtFile. util::Timer virtFileT; virtFileT.start(); - // UberJobs only get one attempt - ProtoRowBuffer::Ptr pRowBuffer = std::make_shared(responseData); - std::string const virtFile = _infileMgr.prepareSrc(pRowBuffer); + auto const csvBuffer = mysql::newCsvStreamBuffer(csvStream); + std::string const virtFile = _infileMgr.prepareSrc(csvBuffer); std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); virtFileT.stop(); - size_t const resultSize = responseData.transmitsize(); + // &&& FOULED_RESULTS + // &&& At this point, it's probably possible to ask csvStream how many bytes were written. + // &&& If 0 bytes were written, the results should be ok and the query doesn't need to be cancelled. + + size_t const resultSize = fileSize; size_t tResultSize; { std::lock_guard resultSzLock(_mtxResultSizeMtx); @@ -226,25 +224,17 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response tct->setSuccess(); tct.reset(); // stop transmit recieve timer before merging happens. - qdisp::CzarStats::get()->addTotalBytesRecv(resultSize); - qdisp::CzarStats::get()->addTotalRowsRecv(responseData.rowcount()); - // Stop here (if requested) after collecting stats on the amount of data collected // from workers. if (_config.debugNoMerge) { return true; } - auto start = CLOCK::now(); - - // Always using MyIsam - ret = _applyMysqlMyIsam(infileStatement, resultSize); - auto end = CLOCK::now(); + auto start = std::chrono::system_clock::now(); + ret = _applyMysqlMyIsam(infileStatement, fileSize); + auto end = std::chrono::system_clock::now(); auto mergeDur = std::chrono::duration_cast(end - start); - LOGS(_log, LOG_LVL_TRACE, - "mergeDur=" << mergeDur.count() << " sema(total=" << _semaMgrConn->getTotalCount() - << " used=" << _semaMgrConn->getUsedCount() << ")"); - + LOGS(_log, LOG_LVL_DEBUG, "mergeDur=" << mergeDur.count()); if (not ret) { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); } diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index aa8085c219..3334c74934 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -43,8 +43,6 @@ #include "util/Error.h" #include "util/EventThread.h" -#include "util/InstanceCount.h" - // Forward declarations namespace lsst::qserv { namespace mysql { @@ -107,8 +105,14 @@ class InfileMerger { InfileMerger& operator=(InfileMerger const&) = delete; ~InfileMerger() = default; + /// Merge a worker response, which contains a single message + /// @return true if merge was successfully imported. + bool merge(proto::ResponseSummary const& responseSummary, + std::shared_ptr const& csvStream); + /// Merge the result data collected over Http. - bool mergeHttp(std::shared_ptr const& uberJob, proto::ResponseData const& responseData); + bool mergeHttp(std::shared_ptr const& uberJob, uint64_t fileSize, + std::shared_ptr const& csvStream); /// Indicate the merge for the job is complete. void mergeCompleteFor(int jobId); diff --git a/src/sql/SqlResults.cc b/src/sql/SqlResults.cc index b96d5d1c85..d7d918ec7b 100644 --- a/src/sql/SqlResults.cc +++ b/src/sql/SqlResults.cc @@ -179,6 +179,27 @@ bool SqlResults::extractFirst4Columns(std::vector& col1, std::vecto return true; } +bool SqlResults::extractFirst6Columns(std::vector& col1, std::vector& col2, + std::vector& col3, std::vector& col4, + std::vector& col5, std::vector& col6, + SqlErrorObject& errObj) { + int i, s = _results.size(); + for (i = 0; i < s; ++i) { + MYSQL_ROW row; + while ((row = mysql_fetch_row(_results[i])) != nullptr) { + col1.push_back(EMPTY_STR_IF_NULL(row[0])); + col2.push_back(EMPTY_STR_IF_NULL(row[1])); + col3.push_back(EMPTY_STR_IF_NULL(row[2])); + col4.push_back(EMPTY_STR_IF_NULL(row[3])); + col5.push_back(EMPTY_STR_IF_NULL(row[4])); + col6.push_back(EMPTY_STR_IF_NULL(row[5])); + } + mysql_free_result(_results[i]); + } + _results.clear(); + return true; +} + std::vector> SqlResults::extractFirstNColumns(size_t numColumns) { std::vector> rows; for (int resultIdx = 0, numResults = _results.size(); resultIdx < numResults; ++resultIdx) { diff --git a/src/sql/SqlResults.h b/src/sql/SqlResults.h index 2c860c6213..f696b9185c 100644 --- a/src/sql/SqlResults.h +++ b/src/sql/SqlResults.h @@ -91,12 +91,16 @@ class SqlResults : boost::noncopyable { unsigned long long getAffectedRows() const { return _affectedRows; } bool extractFirstValue(std::string&, SqlErrorObject&); bool extractFirstColumn(std::vector&, SqlErrorObject&); + // TODO:UJ these extractFirst things are not pretty &&& bool extractFirst2Columns(std::vector&, // FIXME: generalize std::vector&, SqlErrorObject&); bool extractFirst3Columns(std::vector&, // FIXME: generalize std::vector&, std::vector&, SqlErrorObject&); bool extractFirst4Columns(std::vector&, std::vector&, std::vector&, std::vector&, SqlErrorObject&); + bool extractFirst6Columns(std::vector&, std::vector&, std::vector&, + std::vector&, std::vector&, std::vector&, + SqlErrorObject&); template bool extractFirstColumns(SqlErrorObject& err, Columns&... cols) { @@ -104,12 +108,10 @@ class SqlResults : boost::noncopyable { return _extractFirstColumnsImpl(err, columns); } - /** - * Extract a result set into the 2D array. - * @param numColumns The number of columns in the array. - * @return a 2D array, where the first index of the array represents rows - * and the second index represents columns. - */ + /// Extract a result set into the 2D array. + /// @param numColumns The number of columns in the array. + /// @return a 2D array, where the first index of the array represents rows + /// and the second index represents columns. std::vector> extractFirstNColumns(size_t numColumns); void freeResults(); diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index 171ed37448..d3c76b88e4 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -263,15 +263,12 @@ shared_ptr FileChannelShared::create(shared_ptr const& sendChannel, qmeta::CzarId czarId, string const& workerId) - : _isUberJob(false), - _sendChannel(sendChannel), + : _sendChannel(sendChannel), _uberJobId(0), _czarId(czarId), _czarHostName(""), ///< Name of the czar host. _czarPort(-1), - _workerId(workerId), - _protobufArena(make_unique()), - _scsId(scsSeqId++) { + _workerId(workerId) { LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created"); if (_sendChannel == nullptr) { throw util::Bug(ERR_LOC, "FileChannelShared constructor given nullptr"); @@ -288,21 +285,18 @@ FileChannelShared::Ptr FileChannelShared::create(std::shared_ptr const& uberJobData, qmeta::CzarId czarId, string const& czarHostName, int czarPort, string const& workerId) - : _isUberJob(true), - _sendChannel(nullptr), + : _sendChannel(nullptr), _uberJobData(uberJobData), _uberJobId(uberJobData->getUberJobId()), _czarId(czarId), _czarHostName(czarHostName), _czarPort(czarPort), - _workerId(workerId), - _protobufArena(make_unique()), - _scsId(scsSeqId++) { - LOGS(_log, LOG_LVL_TRACE, "FileChannelShared created scsId=" << _scsId << " ujId=" << _uberJobId); + _workerId(workerId) { + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared created ujId=" << _uberJobId); } FileChannelShared::~FileChannelShared() { - LOGS(_log, LOG_LVL_TRACE, "~FileChannelShared scsId=" << _scsId << " ujId=" << _uberJobId); + LOGS(_log, LOG_LVL_TRACE, "~FileChannelShared ujId=" << _uberJobId); // Normally, the channel should not be dead at this time. If it's already // dead it means there was a problem to process a query or send back a response // to Czar. In either case, the file would be useless and it has to be deleted @@ -430,7 +424,6 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr 0 && _rowcount >= ujRowLimit) { // There are enough rows to satisfy the query, so stop reading - hasMoreRows = false; rowLimitComplete = true; LOGS(_log, LOG_LVL_DEBUG, __func__ << " enough rows for query rows=" << _rowcount << " " << task->getIdStr()); @@ -440,7 +433,7 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetIdStr() << " sending done!!!"); } - LOGS(_log, LOG_LVL_TRACE, "FileChannelShared " << task->cName(__func__) << " sending done!!!"); } } transmitT.stop(); @@ -490,7 +482,7 @@ bool FileChannelShared::_kill(lock_guard const& streamMutexLock, string c } void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_ptr const& task, - MYSQL_RES* mResult, uint64_t& bytes, uint32_t& rows, + MYSQL_RES* mResult, uint64_t& bytes, uint64_t& rows, util::MultiError& multiErr) { if (!_file.is_open()) { _fileName = task->getUberJobData()->resultFilePath(); @@ -506,6 +498,7 @@ void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p string const fieldEndsWith = "\t"; string const rowEndsWith = "\n"; string const mysqlNull("\\N"); + int const numFields = mysql_num_fields(mResult); bytes = 0; rows = 0; @@ -534,7 +527,7 @@ void FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p } void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { - LOGS(_log, LOG_LVL_TRACE, "FileChannelShared::_removeFile " << _fileName << " scsId=" << _scsId); + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared::_removeFile " << _fileName); if (!_fileName.empty()) { if (_file.is_open()) { _file.close(); @@ -555,7 +548,7 @@ void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ptr const& task, bool cancelled, util::MultiError const& multiErr, bool mustSend) { auto const queryId = task->getQueryId(); - auto const jobId = task->getJobId(); + auto const jobId = task->getJobId(); // TODO:UJ this should be UberJobId auto const idStr(makeIdStr(queryId, jobId)); // This lock is required for making consistent modifications and usage of the metadata @@ -569,7 +562,7 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ return false; } - // Prepare the response object and serialize in into a message that will + // Prepare the response object and put into a message that will // be sent to the Czar. string httpFileUrl = task->getUberJobData()->resultFileHttpUrl(); auto ujd = _uberJobData.lock(); diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 3ad2bdc8c8..57ab7d2dbd 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -39,7 +39,6 @@ #include "wbase/SendChannel.h" // Forward declarations - namespace lsst::qserv::wbase { class Task; } // namespace lsst::qserv::wbase @@ -196,7 +195,7 @@ class FileChannelShared { * or write into the file. */ void _writeToFile(std::lock_guard const& tMtxLock, std::shared_ptr const& task, - MYSQL_RES* mResult, std::uint64_t& bytes, std::uint32_t& rows, + MYSQL_RES* mResult, std::uint64_t& bytes, std::uint64_t& rows, util::MultiError& multiErr); /// Write a string into the currently open file. diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 07a7df6673..0a805ae43c 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -38,7 +38,8 @@ #include "http/Method.h" #include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" -#include "protojson/JobReadyMsg.h" +#include "protojson/UberJobErrorMsg.h" +#include "protojson/UberJobReadyMsg.h" #include "util/Bug.h" #include "util/MultiError.h" #include "wconfig/WorkerConfig.h" @@ -109,11 +110,13 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount cName(__func__) << " _foreman was null, which should only happen in unit tests"); } - auto repliInstId = wconfig::WorkerConfig::instance()->replicationInstanceId(); - auto repliAuthKey = wconfig::WorkerConfig::instance()->replicationAuthKey(); - auto jrMsg = protojson::JobReadyMsg::create(repliInstId, repliAuthKey, workerIdStr, _czarName, _czarId, - _queryId, _uberJobId, httpFileUrl, rowCount, fileSize); - json request = jrMsg->serializeJson(); + auto const& repliInstId = wconfig::WorkerConfig::instance()->replicationInstanceId(); + auto const& repliAuthKey = wconfig::WorkerConfig::instance()->replicationAuthKey(); + unsigned int const version = http::MetaModule::version; + auto jrMsg = protojson::UberJobReadyMsg::create(repliInstId, repliAuthKey, version, workerIdStr, + _czarName, _czarId, _queryId, _uberJobId, httpFileUrl, + rowCount, fileSize); + json request = jrMsg->toJson(); auto const method = http::Method::POST; vector const headers = {"Content-Type: application/json"}; @@ -150,15 +153,14 @@ void UberJobData::responseError(util::MultiError& multiErr, int chunkId, bool ca LOGS(_log, logLvl, errorMsg); } - json request = {{"version", http::MetaModule::version}, - {"workerid", _foreman->chunkInventory()->id()}, - {"auth_key", _authKey}, - {"czar", _czarName}, - {"czarid", _czarId}, - {"queryid", _queryId}, - {"uberjobid", _uberJobId}, - {"errorCode", errorCode}, - {"errorMsg", errorMsg}}; + string const workerIdStr = _foreman->chunkInventory()->id(); + auto repliInstId = wconfig::WorkerConfig::instance()->replicationInstanceId(); + auto repliAuthKey = wconfig::WorkerConfig::instance()->replicationAuthKey(); + unsigned int const version = http::MetaModule::version; + auto jrMsg = + protojson::UberJobErrorMsg::create(repliInstId, repliAuthKey, version, workerIdStr, _czarName, + _czarId, _queryId, _uberJobId, errorCode, errorMsg); + json request = jrMsg->toJson(); auto const method = http::Method::POST; vector const headers = {"Content-Type: application/json"}; @@ -190,7 +192,7 @@ void UberJobData::_queueUJResponse(http::Method method_, std::vectorgetFqdn() + ":" + to_string(_resultsHttpPort) + "/" + _resultFileName(); } diff --git a/src/wcontrol/WCzarInfoMap.cc b/src/wcontrol/WCzarInfoMap.cc index 0ccef5a787..9a75343c83 100644 --- a/src/wcontrol/WCzarInfoMap.cc +++ b/src/wcontrol/WCzarInfoMap.cc @@ -118,7 +118,7 @@ void WCzarInfo::_sendMessage() { vector const headers = {"Content-Type: application/json"}; string const url = "http://" + czInfo->czHostName + ":" + to_string(czInfo->czPort) + "/workerczarcomissue"; - auto jsReqPtr = _workerCzarComIssue->serializeJson(); + auto jsReqPtr = _workerCzarComIssue->toJson(); uniLock.unlock(); // Must unlock before communication auto requestStr = jsReqPtr->dump(); diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index e497aaa0db..6ceb1af217 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -126,8 +126,7 @@ bool QueryRunner::_initConnection() { bool QueryRunner::runQuery() { util::HoldTrack::Mark runQueryMarkA(ERR_LOC, "runQuery " + to_string(_task->getQueryId())); QSERV_LOGCONTEXT_QUERY_JOB(_task->getQueryId(), _task->getJobId()); - LOGS(_log, LOG_LVL_TRACE, - "QueryRunner " << _task->cName(__func__) << " scsId=" << _task->getSendChannel()->getScsId()); + LOGS(_log, LOG_LVL_TRACE, "QueryRunner " << _task->cName(__func__)); // Start tracking the task. auto now = chrono::system_clock::now(); diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 0201283d13..4e4bb36a08 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -181,6 +181,7 @@ BOOST_AUTO_TEST_CASE(Simple) { MsgInfo mInfo; auto msgJson = newTaskJson(mInfo); shared_ptr sendC(SendChannel::newNopChannel()); + auto sChannel = FileChannelShared::create(sendC, mInfo.czarId); FakeBackend::Ptr backend = make_shared(); shared_ptr crm = ChunkResourceMgr::newMgr(backend); diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index 304653b9bf..fb87bf3db8 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -206,7 +206,6 @@ void ScanScheduler::queCmd(vector const& cmds) { } } - tsk->setMemMan(_memMan); tasks.push_back(tsk); LOGS(_log, LOG_LVL_TRACE, getName() << " queCmd " << tsk->getIdStr()); } diff --git a/src/xrdsvc/SsiService.cc b/src/xrdsvc/SsiService.cc index 40c85c4a8f..3f3335bb63 100644 --- a/src/xrdsvc/SsiService.cc +++ b/src/xrdsvc/SsiService.cc @@ -48,8 +48,6 @@ #include "http/Client.h" #include "http/MetaModule.h" #include "http/Method.h" -#include "memman/MemMan.h" -#include "memman/MemManNone.h" #include "mysql/MySqlConfig.h" #include "mysql/MySqlConnection.h" #include "qhttp/Server.h" @@ -157,25 +155,6 @@ SsiService::SsiService(XrdSsiLogger* log) { throw wconfig::WorkerConfigError("Unable to connect to MySQL"); } auto const workerConfig = wconfig::WorkerConfig::instance(); - string cfgMemMan = workerConfig->getMemManClass(); - memman::MemMan::Ptr memMan; - if (cfgMemMan == "MemManReal") { - // Default to 1 gigabyte - uint64_t memManSize = workerConfig->getMemManSizeMb() * 1000000; - LOGS(_log, LOG_LVL_DEBUG, - "Using MemManReal with memManSizeMb=" << workerConfig->getMemManSizeMb() - << " location=" << workerConfig->getMemManLocation()); - memMan = shared_ptr( - memman::MemMan::create(memManSize, workerConfig->getMemManLocation())); - } else if (cfgMemMan == "MemManNone") { - memMan = make_shared(1, false); - } else if (cfgMemMan == "MemManNoneRelaxed") { - bool const alwaysLock = true; - memMan = make_shared(1, alwaysLock); - } else { - LOGS(_log, LOG_LVL_ERROR, "Unrecognized memory manager " << cfgMemMan); - throw wconfig::WorkerConfigError("Unrecognized memory manager."); - } // Set thread pool size. unsigned int poolSize = ranges::max({wsched::BlendScheduler::getMinPoolSize(), @@ -204,20 +183,20 @@ SsiService::SsiService(XrdSsiLogger* log) { vector scanSchedulers{ make_shared("SchedSlow", maxThread, workerConfig->getMaxReserveSlow(), workerConfig->getPrioritySlow(), - workerConfig->getMaxActiveChunksSlow(), memMan, medium + 1, - slow, slowScanMaxMinutes), + workerConfig->getMaxActiveChunksSlow(), medium + 1, slow, + slowScanMaxMinutes), make_shared("SchedFast", maxThread, workerConfig->getMaxReserveFast(), workerConfig->getPriorityFast(), - workerConfig->getMaxActiveChunksFast(), memMan, fastest, fast, + workerConfig->getMaxActiveChunksFast(), fastest, fast, fastScanMaxMinutes), make_shared( "SchedMed", maxThread, workerConfig->getMaxReserveMed(), workerConfig->getPriorityMed(), - workerConfig->getMaxActiveChunksMed(), memMan, fast + 1, medium, medScanMaxMinutes), + workerConfig->getMaxActiveChunksMed(), fast + 1, medium, medScanMaxMinutes), }; auto snail = make_shared( "SchedSnail", maxThread, workerConfig->getMaxReserveSnail(), workerConfig->getPrioritySnail(), - workerConfig->getMaxActiveChunksSnail(), memMan, slow + 1, slowest, snailScanMaxMinutes); + workerConfig->getMaxActiveChunksSnail(), slow + 1, slowest, snailScanMaxMinutes); wpublish::QueriesAndChunks::Ptr queries = wpublish::QueriesAndChunks::setupGlobal( chrono::minutes(5), chrono::minutes(2), maxTasksBootedPerUserQuery, maxConcurrentBootedTasks, From ccc7c0bda92b57d63f6c27003a01d729ca50fcc9 Mon Sep 17 00:00:00 2001 From: John Gates Date: Fri, 9 May 2025 12:35:33 -0700 Subject: [PATCH 11/15] Removed protobufs. --- src/CMakeLists.txt | 1 - src/ccontrol/CMakeLists.txt | 2 - src/ccontrol/MergingHandler.cc | 228 +++------------------------- src/ccontrol/MergingHandler.h | 22 +-- src/ccontrol/UserQuerySelect.cc | 1 - src/czar/CMakeLists.txt | 1 - src/czar/Czar.cc | 7 +- src/mysql/CMakeLists.txt | 1 - src/mysql/CsvBuffer.cc | 3 + src/mysql/CsvBuffer.h | 6 + src/mysql/RowBuffer.h | 85 ----------- src/mysql/testMySqlUtils.cc | 1 - src/proto/CMakeLists.txt | 23 --- src/proto/COPYRIGHT | 1 - src/proto/FrameBuffer.cc | 110 -------------- src/proto/FrameBuffer.h | 253 -------------------------------- src/proto/ProtoHeaderWrap.cc | 45 ------ src/proto/ProtoHeaderWrap.h | 47 ------ src/proto/worker.proto | 68 --------- src/qana/CMakeLists.txt | 1 - src/qdisp/CMakeLists.txt | 1 - src/qdisp/JobDescription.cc | 1 - src/qdisp/ResponseHandler.h | 36 ++--- src/qdisp/UberJob.cc | 19 +-- src/qdisp/testQDisp.cc | 5 +- src/qmeta/CMakeLists.txt | 1 - src/qproc/CMakeLists.txt | 1 - src/query/CMakeLists.txt | 1 - src/rproc/CMakeLists.txt | 1 - src/rproc/InfileMerger.cc | 45 +++--- src/rproc/InfileMerger.h | 9 +- src/sql/SqlResults.cc | 86 ++++------- src/sql/SqlResults.h | 39 +++-- src/util/ResultFileName.cc | 40 ++--- src/util/ResultFileName.h | 36 ++--- src/util/testResultFileName.cc | 30 ++-- src/wbase/Base.h | 9 +- src/wbase/CMakeLists.txt | 1 - src/wbase/FileChannelShared.cc | 2 - src/wbase/FileChannelShared.h | 8 +- src/wbase/SendChannel.cc | 1 - src/wbase/Task.cc | 1 - src/wbase/TaskState.h | 4 - src/wbase/UberJobData.cc | 12 +- src/wconfig/CMakeLists.txt | 2 - src/wconfig/WorkerConfig.h | 51 ------- src/wcontrol/CMakeLists.txt | 1 - src/wdb/CMakeLists.txt | 1 - src/wdb/ChunkResource.h | 9 +- src/wdb/QueryRunner.cc | 3 - src/wdb/testQueryRunner.cc | 2 - src/wpublish/CMakeLists.txt | 2 - src/wsched/BlendScheduler.cc | 1 - src/wsched/CMakeLists.txt | 1 - src/wsched/GroupScheduler.cc | 1 - src/wsched/testSchedulers.cc | 4 - 56 files changed, 215 insertions(+), 1158 deletions(-) delete mode 100644 src/mysql/RowBuffer.h delete mode 100644 src/proto/CMakeLists.txt delete mode 100644 src/proto/COPYRIGHT delete mode 100644 src/proto/FrameBuffer.cc delete mode 100644 src/proto/FrameBuffer.h delete mode 100644 src/proto/ProtoHeaderWrap.cc delete mode 100644 src/proto/ProtoHeaderWrap.h delete mode 100644 src/proto/worker.proto diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 822a8ae3b5..70540f15a2 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -66,7 +66,6 @@ add_subdirectory(http) add_subdirectory(mysql) add_subdirectory(parser) add_subdirectory(partition) -add_subdirectory(proto) add_subdirectory(protojson) add_subdirectory(proxy) add_subdirectory(qana) diff --git a/src/ccontrol/CMakeLists.txt b/src/ccontrol/CMakeLists.txt index 9d54d33c17..a732c4bdd9 100644 --- a/src/ccontrol/CMakeLists.txt +++ b/src/ccontrol/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(ccontrol SHARED) -add_dependencies(ccontrol proto) target_include_directories(ccontrol PRIVATE ${ANTLR4_INCLUDE_DIR} @@ -29,7 +28,6 @@ target_link_libraries(ccontrol PUBLIC css log parser - proto sphgeom ) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 66df1aeff1..935206440c 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -46,8 +46,6 @@ #include "http/ClientConnPool.h" #include "http/Method.h" #include "mysql/CsvBuffer.h" -#include "proto/ProtoHeaderWrap.h" -#include "proto/worker.pb.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" @@ -145,106 +143,6 @@ string readHttpFileAndMerge(lsst::qserv::qdisp::UberJob::Ptr const& uberJob, str ", offset: " + to_string(offset) + ", ex: " + string(ex.what()); LOGS(_log, LOG_LVL_ERROR, context << errMsg); return errMsg; - /* &&& - // A value of the flag is set by the message processor when it's time to finish - // or abort reading the file. - bool last = false; - char const* next = inBuf; - char const* const end = inBuf + inBufSize; - LOGS(_log, LOG_LVL_TRACE, context << " next=" << (uint64_t)next << " end=" << (uint64_t)end); - while ((next < end) && !last) { - if (exec->getCancelled()) { - throw runtime_error(context + " query was cancelled"); - } - if (msgSizeBytes == 0) { - // Continue or finish reading the frame header. - size_t const bytes2read = - std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); - std::memcpy(msgSizeBuf.data() + msgSizeBufNext, next, bytes2read); - next += bytes2read; - offset += bytes2read; - msgSizeBufNext += bytes2read; - if (msgSizeBufNext == sizeof(uint32_t)) { - ++headerCount; - // Done reading the frame header. - msgSizeBufNext = 0; - // Parse and evaluate the message length. - msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); - if (msgSizeBytes == 0) { - throw runtime_error("message size is 0 at offset " + - to_string(offset - sizeof(uint32_t)) + ", file: " + - httpUrl); - } - if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { - throw runtime_error("message size " + to_string(msgSizeBytes) + " at offset " - + to_string(offset - sizeof(uint32_t)) + " exceeds the hard limit of " + - to_string(ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) + - ", file: " + httpUrl); - } - // Extend the message buffer (if needed). Note that buffer never gets - // truncated to avoid excessive memory deallocations/allocations. - if (msgBufSize < msgSizeBytes) { - msgBufSize = msgSizeBytes; - msgBuf.reset(new char[msgBufSize]); - } - // Starts the tracker to measure the performance of the network I/O. - transmitRateTracker = - make_unique>(reportFileRecvRate); - } - } else { - // Continue or finish reading the message body. - size_t const bytes2read = - std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); - std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); - next += bytes2read; - offset += bytes2read; - msgBufNext += bytes2read; - if (msgBufNext == msgSizeBytes) { - // Done reading message body. - msgBufNext = 0; - - // Destroying the tracker will result in stopping the tracker's timer and - // reporting the file read rate before proceeding to the merge. - if (transmitRateTracker != nullptr) { - transmitRateTracker->addToValue(msgSizeBytes); - transmitRateTracker->setSuccess(); - transmitRateTracker.reset(); - } - - // Parse and evaluate the message. - mergeHappened = true; - bool messageReadyResult = messageIsReady(msgBuf.get(), msgSizeBytes, last); - totalBytesRead += msgSizeBytes; - if (!messageReadyResult) { - success = false; - throw runtime_error("message processing failed at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - // Reset the variable to prepare for reading the next header & message (if any). - msgSizeBytes = 0; - } else { - LOGS(_log, LOG_LVL_TRACE, - context << " headerCount=" << headerCount - << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); - } - } - } - }); - LOGS(_log, LOG_LVL_TRACE, - context << " headerCount=" << headerCount << " msgSizeBytes=" << msgSizeBytes - << " totalBytesRead=" << totalBytesRead); - if (msgSizeBufNext != 0) { - throw runtime_error("short read of the message header at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - if (msgBufNext != 0) { - throw runtime_error("short read of the message body at offset " + - to_string(offset - msgSizeBytes) + ", file: " + httpUrl); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); - success = false; - */ } // Remove the file from the worker if it still exists. Report and ignore errors. @@ -337,28 +235,20 @@ std::ostream& MergingHandler::print(std::ostream& os) const { return os << "MergingRequester(flushed=" << (_flushed ? "true)" : "false)"); } -bool queryIsNoLongerActive(qdisp::UberJob::Ptr const& uberJob) { //&&& - // Do nothing if the query got cancelled for any reason. - if (uberJob->isQueryCancelled()) return true; - - // Check for other indicators that the query may have cancelled or finished. - auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { - return true; - } - return false; -} - -bool MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uberJob, string const& fileUrl, - uint64_t fileSize) { +qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uberJob, string const& fileUrl, + uint64_t fileSize) { if (_flushed) { throw util::Bug(ERR_LOC, "already flushed"); } - if (fileSize == 0) return true; + if (fileSize == 0) { + return qdisp::MergeEndStatus(true); + } // After this final test the job's result processing can't be interrupted. - if (uberJob->isQueryCancelled()) return true; + if (uberJob->isQueryCancelled()) { + return qdisp::MergeEndStatus(true); + } // Read from the http stream and push records into the CSV stream in a separate thread. // Note the fixed capacity of the stream which allows up to 2 records to be buffered @@ -385,7 +275,7 @@ bool MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uberJob, string const }, MergingHandler::_getHttpConnPool()); // Push the stream terminator to indicate the end of the stream. - // It may be neeeded to unblock the table merger which may be still attempting to read + // It may be needed to unblock the table merger which may be still attempting to read // from the CSV stream. if (!fileReadErrorMsg.empty()) { csvStream->push(nullptr, 0); @@ -406,61 +296,16 @@ bool MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uberJob, string const _setError(ccontrol::MSG_HTTP_RESULT, fileReadErrorMsg, util::ErrorCode::RESULT_IMPORT); } _flushed = true; - return fileMergeSuccess && fileReadErrorMsg.empty(); -} - -void MergingHandler::_setError(int code, std::string const& msg, int errorState) { - LOGS(_log, LOG_LVL_DEBUG, "_setError: code: " << code << ", message: " << msg); - auto exec = _executive.lock(); - if (exec == nullptr) return; - exec->addMultiError(code, msg, errorState); -} - -tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t fileSize, uint64_t expectedRows, - uint64_t& resultRows) { - bool success = false; - bool shouldCancel = false; - - // This is needed to ensure the job query would be staying alive for the duration - // of the operation to prevent inconsistency within the application. - auto const uberJob = getUberJob().lock(); - if (uberJob == nullptr) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, uberJob was NULL"); - return {success, shouldCancel}; // both should still be false - } - LOGS(_log, LOG_LVL_TRACE, - "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); - - success = _mergeHttp(uberJob, fileUrl, fileSize); - // &&& FOULED_RESULTS need to do something about shouldCancel. - // &&& until there is some way to know if csvStream has merged any bytes, just assume it has fouled the - // results. - if (!success) shouldCancel = true; - - if (!success || shouldCancel) { - LOGS(_log, LOG_LVL_WARN, __func__ << " success=" << success << " shouldCancel=" << shouldCancel); - } - - if (success) { - _infileMerger->mergeCompleteFor(uberJob->getUjId()); - } - return {success, shouldCancel}; -} - -/* &&& -bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, - proto::ResponseData const& responseData) { - if (_flushed) { - throw util::Bug(ERR_LOC, "already flushed"); - } - bool const success = _infileMerger->mergeHttp(uberJob, responseData); - if (!success) { - LOGS(_log, LOG_LVL_WARN, __func__ << " failed"); - util::Error const& err = _infileMerger->getError(); - _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg(), util::ErrorCode::RESULT_IMPORT); + qdisp::MergeEndStatus mergeEStatus(fileMergeSuccess && fileReadErrorMsg.empty()); + if (!mergeEStatus.success) { + // This error check needs to come after the csvThread.join() to avoid race conditions. + if (csvStream->getBytesWritten() > 0) { + // There was a failure and bytes were written, result table is ruined. + mergeEStatus.contaminated = true; + } } - return success; + return mergeEStatus; } void MergingHandler::_setError(int code, std::string const& msg, int errorState) { @@ -470,54 +315,25 @@ void MergingHandler::_setError(int code, std::string const& msg, int errorState) exec->addMultiError(code, msg, errorState); } -tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expectedRows, - uint64_t& resultRows) { - bool success = false; - bool shouldCancel = false; - +qdisp::MergeEndStatus MergingHandler::flushHttp(string const& fileUrl, uint64_t fileSize) { // This is needed to ensure the job query would be staying alive for the duration // of the operation to prevent inconsistency within the application. auto const uberJob = getUberJob().lock(); if (uberJob == nullptr) { LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, uberJob was NULL"); - return {success, shouldCancel}; // both should still be false + return qdisp::MergeEndStatus(false); } LOGS(_log, LOG_LVL_TRACE, "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); - // Dispatch result processing to the corresponidng method which depends on - // the result delivery protocol configured at the worker. - // Notify the file reader when all rows have been read by setting 'last = true'. - auto const dataMergerHttp = [&](char const* buf, uint32_t bufSize, bool& last) { - LOGS(_log, LOG_LVL_TRACE, "dataMergerHttp"); - last = true; - proto::ResponseData responseData; - if (responseData.ParseFromArray(buf, bufSize) && responseData.IsInitialized()) { - bool const mergeSuccess = _mergeHttp(uberJob, responseData); - if (mergeSuccess) { - resultRows += responseData.row_size(); - last = resultRows >= expectedRows; - } - return mergeSuccess; - } - throw runtime_error("MergingHandler::flush ** message deserialization failed **"); - }; - - tie(success, shouldCancel) = - ::readHttpFileAndMergeHttp(uberJob, fileUrl, dataMergerHttp, MergingHandler::_getHttpConnPool()); - - if (!success || shouldCancel) { - LOGS(_log, LOG_LVL_WARN, __func__ << " success=" << success << " shouldCancel=" << shouldCancel); - } - - if (success) { + qdisp::MergeEndStatus mergeStatus = _mergeHttp(uberJob, fileUrl, fileSize); + if (mergeStatus.success) { _infileMerger->mergeCompleteFor(uberJob->getUjId()); } - return {success, shouldCancel}; + return mergeStatus; } ->>>>>>> a27525c04017db9a30061fa0bb4b5228c0c5d1b2 */ void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int errState) { if (!_errorSet.exchange(true)) { _setError(errorCode, errorMsg, errState); diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 252ecc74bd..c27070831c 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -32,15 +32,14 @@ #include "qdisp/ResponseHandler.h" // Forward declarations - -namespace lsst::qserv::proto { -class ResponseData; -class ResponseSummary; -} // namespace lsst::qserv::proto +namespace lsst::qserv::http { +class ClientConnPool; +} // namespace lsst::qserv::http namespace lsst::qserv::qdisp { class Executive; class JobQuery; +class MergeEndStatus; class UberJob; } // namespace lsst::qserv::qdisp @@ -70,12 +69,8 @@ class MergingHandler : public qdisp::ResponseHandler { /// @see ResponseHandler::flushHttp /// @see MerginHandler::_mergeHttp - std::tuple flushHttp( - std::string const& fileUrl, uint64_t fileSize, uint64_t expectedRows, - /* &&& - std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, - >>>>>>> a27525c04017db9a30061fa0bb4b5228c0c5d1b2 */ - uint64_t& resultRows) override; + /// @see qdisp::MergeEndStatus + qdisp::MergeEndStatus flushHttp(std::string const& fileUrl, uint64_t fileSize) override; /// @see ResponseHandler::flushHttpError void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; @@ -88,9 +83,8 @@ class MergingHandler : public qdisp::ResponseHandler { private: /// Call InfileMerger to do the work of merging this data to the result. - - bool _mergeHttp(std::shared_ptr const& uberJob, std::string const& fileUrl, - uint64_t fileSize); + qdisp::MergeEndStatus _mergeHttp(std::shared_ptr const& uberJob, + std::string const& fileUrl, uint64_t fileSize); /// Set error code and string. void _setError(int code, std::string const& msg, int errorState); diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index a1b8faf6ef..68aa2f324f 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -84,7 +84,6 @@ #include "czar/CzarRegistry.h" #include "global/constants.h" #include "global/LogContext.h" -#include "proto/worker.pb.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" #include "qmeta/MessageStore.h" diff --git a/src/czar/CMakeLists.txt b/src/czar/CMakeLists.txt index 1257769cc6..658b49780f 100644 --- a/src/czar/CMakeLists.txt +++ b/src/czar/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(czar SHARED) -add_dependencies(czar proto) target_sources(czar PRIVATE ActiveWorker.cc diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 3a5a074306..4af93623a2 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -49,7 +49,10 @@ #include "czar/MessageTable.h" #include "czar/CzarRegistry.h" #include "global/LogContext.h" -#include "proto/worker.pb.h" +#include "http/Client.h" +#include "http/ClientConnPool.h" +#include "http/MetaModule.h" +#include "http/Method.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qproc/DatabaseModels.h" @@ -538,6 +541,7 @@ SubmitResult Czar::getQueryInfo(QueryId queryId) const { ", sql=" + sql; throw runtime_error(msg); } + vector colStatus; vector colCzarId; vector colCzarType; @@ -558,6 +562,7 @@ SubmitResult Czar::getQueryInfo(QueryId queryId) const { ", sql=" + sql; throw runtime_error(msg); } + if (colStatus.size() != 1) { string const msg = context + "Unknown user query, err=" + err.printErrMsg() + ", sql=" + sql; throw runtime_error(msg); diff --git a/src/mysql/CMakeLists.txt b/src/mysql/CMakeLists.txt index fd5435b75a..2796872566 100644 --- a/src/mysql/CMakeLists.txt +++ b/src/mysql/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(mysql SHARED) -add_dependencies(mysql proto) target_sources(mysql PRIVATE CsvBuffer.cc diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index fcd58372df..ae5a9f6b1d 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -303,6 +303,8 @@ class CsvStreamBuffer : public CsvBuffer { public: explicit CsvStreamBuffer(std::shared_ptr const& csvStream) : _csvStream(csvStream) {} + ~CsvStreamBuffer() override = default; + unsigned fetch(char* buffer, unsigned bufLen) override { if (bufLen == 0) { throw LocalInfileError("CsvStreamBuffer::fetch Can't fetch non-positive bytes"); @@ -320,6 +322,7 @@ class CsvStreamBuffer : public CsvBuffer { unsigned const bytesToCopy = std::min(bufLen, static_cast(_str->size() - _offset)); ::memcpy(buffer, _str->data() + _offset, bytesToCopy); _offset += bytesToCopy; + _csvStream->increaseBytesWrittenBy(bytesToCopy); return bytesToCopy; } diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index 730676dca2..4803a395d5 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -43,6 +43,8 @@ namespace lsst::qserv::mysql { */ class CsvBuffer { public: + virtual ~CsvBuffer() = default; + /// Fetch a number of bytes into a buffer. Return the number of bytes /// fetched. Returning less than bufLen does NOT indicate EOF. virtual unsigned fetch(char* buffer, unsigned bufLen) = 0; @@ -111,6 +113,9 @@ class CsvStream { */ bool empty() const; + void increaseBytesWrittenBy(size_t bytesToCopy) { _bytesWritten += bytesToCopy; } + size_t getBytesWritten() const { return _bytesWritten; } + private: CsvStream(std::size_t maxRecords); @@ -118,6 +123,7 @@ class CsvStream { std::condition_variable _cv; std::size_t const _maxRecords; std::list> _records; + std::atomic _bytesWritten; }; /** diff --git a/src/mysql/RowBuffer.h b/src/mysql/RowBuffer.h deleted file mode 100644 index 389790da9c..0000000000 --- a/src/mysql/RowBuffer.h +++ /dev/null @@ -1,85 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_MYSQL_ROWBUFFER_H -#define LSST_QSERV_MYSQL_ROWBUFFER_H - -// System headers -#include -#include -#include - -// Third-party headers -#include - -namespace lsst::qserv::proto { -class Result; -} // namespace lsst::qserv::proto - -namespace lsst::qserv::mysql { - -/// Row is a mysql row abstraction that bundles field sizes and counts. Row is -/// shallow, and does not perform any memory management. -struct Row { - Row() : row(nullptr), lengths(nullptr), numFields(-1) {} - virtual ~Row(); - - // Shallow copies all-around. - Row(char** row_, unsigned long int* lengths_, int numFields_) - : row(row_), lengths(lengths_), numFields(numFields_) {} - - unsigned int minRowSize() const { - unsigned int sum = 0; - for (int i = 0; i < numFields; ++i) { - sum += lengths[i]; - } - return sum; - } - - char** row; - unsigned long int* lengths; - int numFields; -}; - -/// RowBuffer: an buffer from which arbitrarily-sized buckets of bytes -/// can be read. The buffer represents a tab-separated-field, -/// line-delimited-tuple sequence of tuples. -class RowBuffer { -public: - typedef std::shared_ptr Ptr; - - virtual ~RowBuffer(); - - /// Fetch a number of bytes into a buffer. Return the number of bytes - /// fetched. Returning less than bufLen does NOT indicate EOF. - virtual unsigned fetch(char* buffer, unsigned bufLen) = 0; - - /// Construct a RowBuffer tied to a MySQL query result - static Ptr newResRowBuffer(MYSQL_RES* result); - - /// Return a descriptive string. - virtual std::string dump() const = 0; -}; - -} // namespace lsst::qserv::mysql -#endif // LSST_QSERV_MYSQL_ROWBUFFER_H diff --git a/src/mysql/testMySqlUtils.cc b/src/mysql/testMySqlUtils.cc index c8725f7864..f7185388cb 100644 --- a/src/mysql/testMySqlUtils.cc +++ b/src/mysql/testMySqlUtils.cc @@ -29,7 +29,6 @@ #include namespace test = boost::test_tools; -// namespace gio = google::protobuf::io; struct Fixture { Fixture(void) {} diff --git a/src/proto/CMakeLists.txt b/src/proto/CMakeLists.txt deleted file mode 100644 index 2e6e55ab12..0000000000 --- a/src/proto/CMakeLists.txt +++ /dev/null @@ -1,23 +0,0 @@ -add_library(proto SHARED) - -target_compile_options(proto PRIVATE - -Wno-maybe-uninitialized -) - -protobuf_generate_cpp(PROTO_PB_SRCS PROTO_PB_HDRS worker.proto) - -target_sources(proto PRIVATE - ${PROTO_PB_SRCS} - ${PROTO_PB_HDRS} - FrameBuffer.cc - ProtoHeaderWrap.cc -) - -target_link_libraries(proto PUBLIC - log - protobuf -) - -install( - TARGETS proto -) diff --git a/src/proto/COPYRIGHT b/src/proto/COPYRIGHT deleted file mode 100644 index b457fc9841..0000000000 --- a/src/proto/COPYRIGHT +++ /dev/null @@ -1 +0,0 @@ -2011-2015 LSST Corp, 2015-2018 SLAC diff --git a/src/proto/FrameBuffer.cc b/src/proto/FrameBuffer.cc deleted file mode 100644 index 798908f9a6..0000000000 --- a/src/proto/FrameBuffer.cc +++ /dev/null @@ -1,110 +0,0 @@ -/* - * LSST Data Management System - * Copyright 2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "proto/FrameBuffer.h" - -// System headers -#include - -// Qserv headers - -namespace lsst::qserv::proto { - -/////////////////////////// -// Class FrameBufferView // -/////////////////////////// - -FrameBufferView::FrameBufferView(char const* data, size_t size) : _data(data), _next(data), _size(size) {} - -uint32_t FrameBufferView::parseLength() { - uint32_t const headerLength = sizeof(uint32_t); - if (_size - (_next - _data) < headerLength) - FrameBufferError( - "FrameBufferView::parseLength() ** not enough data to be interpreted as the frame header " - "**"); - - uint32_t const messageLength = ntohl(*(reinterpret_cast(_next))); - - // Move the pointer to the next message (if any) - _next += headerLength; - - return messageLength; -} - -/////////////////////// -// Class FrameBuffer // -/////////////////////// - -const size_t FrameBuffer::DEFAULT_SIZE = 1024; -const size_t FrameBuffer::DESIRED_LIMIT = 2000000; -const size_t FrameBuffer::HARD_LIMIT = 64000000; - -FrameBuffer::FrameBuffer(size_t capacity) : _data(new char[capacity]), _capacity(capacity), _size(0) { - if (_capacity > HARD_LIMIT) - throw FrameBufferError( - "FrameBuffer::FrameBuffer() ** requested capacity " + std::to_string(capacity) + - " exceeds the hard limit of Google protobuf: " + std::to_string(HARD_LIMIT) + " **"); -} - -FrameBuffer::~FrameBuffer() { - delete[] _data; - _data = 0; - _capacity = 0; - _size = 0; -} - -void FrameBuffer::resize(size_t newSizeBytes) { - // Make sure there is enough space in the buffer to accomodate - // the request. - - extend(newSizeBytes); - - _size = newSizeBytes; -} - -void FrameBuffer::extend(size_t newCapacityBytes) { - if (newCapacityBytes <= _capacity) return; - - // Allocate a larger buffer - - if (newCapacityBytes > HARD_LIMIT) - throw FrameBufferError( - "FrameBuffer::extend() ** requested capacity " + std::to_string(newCapacityBytes) + - " exceeds the hard limit of Google protobuf " + std::to_string(HARD_LIMIT) + " **"); - - char* ptr = new char[newCapacityBytes]; - if (!ptr) - throw FrameBufferError("FrameBuffer::extend() ** failed to allocate a buffer of requested size " + - std::to_string(newCapacityBytes) + " **"); - - // Carry over the meaningful content of the older buffer into the new one - // before disposing the old buffer. - std::copy(_data, _data + _size, ptr); - - delete[] _data; - _data = ptr; - - _capacity = newCapacityBytes; -} - -} // namespace lsst::qserv::proto diff --git a/src/proto/FrameBuffer.h b/src/proto/FrameBuffer.h deleted file mode 100644 index 8cb90716d1..0000000000 --- a/src/proto/FrameBuffer.h +++ /dev/null @@ -1,253 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2018 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_PROTO_FRAME_BUFFER_H -#define LSST_QSERV_PROTO_FRAME_BUFFER_H - -/// FrameBuffer.h declares: -/// -/// struct FrameBufferError -/// class FrameBufferView -/// class FrameBuffer -/// -/// (see individual class documentation for more information) - -// System headers -#include // ntohl -#include // uint32_t -#include -#include - -// Qserv headers - -// Forward declarations - -// This header declarations - -namespace lsst::qserv::proto { - -/** - * Class FrameBufferError is used for throwing exceptions on various - * ubnormal conditions seen in the implementations of the buffer - * classes. - */ -struct FrameBufferError : std::runtime_error { - /// Normal constructor of the exception class - FrameBufferError(std::string const& msg) : std::runtime_error(msg) {} -}; - -/** - * The helper class encapsulating deserialization operations with Google - * protobuf objects on a staticly definied input byte stream. The stream - * is expected to contain an arbitrary sequence of the following pairs of - * records: - * - * 4-bytes: frame header containing 'N' - the length of a message - * N-bytes: the message serialized as a Protobuf object - * ... - */ -class FrameBufferView { -public: - /** - * Construct the buffer - * - * @param data - pointer to the data blob to be parsed - * @param size - the length (bytes) in the data blob - */ - explicit FrameBufferView(char const* data, size_t size); - - // Default construction and copy semantics are proxibited - - FrameBufferView() = delete; - FrameBufferView(FrameBufferView const&) = delete; - FrameBufferView& operator=(FrameBufferView const&) = delete; - - /// Destructor - ~FrameBufferView() = default; - - /* - * Parse and deserialize the message given the specified size of - * the message as informed by a prior frame header. - * If successful the method will also advance the current pointer within - * the data blob past the parsed message thus allowing it to parse the next - * message. - * - * The method will throw exception FrameBufferError if: - * - the buffer doesn't have enough data to be interpreted as - * the message of the required size - * - message deserialization failed - * - * @param message - protobuf object to be initialized upon a successful - * completion of the operation - */ - template - void parse(T& message) { - uint32_t const messageLength = parseLength(); - - if (_size - (_next - _data) < messageLength) - throw FrameBufferError("FrameBufferView::parse() ** not enough data (" + - std::to_string(_size - (_next - _data)) + " bytes instead of " + - std::to_string(messageLength) + " to be interpreted as the message"); - - if (not message.ParseFromArray(_next, messageLength) || not message.IsInitialized()) { - throw FrameBufferError("FrameBufferView::parse() ** message deserialization failed **"); - } - - // Move the pointer to the next message (if any) - _next += messageLength; - } - -private: - /** - * Parse and deserialize the length of a message from the frame header - * at a curren position of the data pointer. - * If succeeded the method will also advance the current pointer within - * the data blob past the parsed message thus allowing to parse the next - * message. - * - * The method will throw one of these exceptions: - * - * std::underflow_error - * if the buffer doesn't have enough data to be interpreted as the - * frame header - * - * @return the length (bytes) of of the next message - */ - uint32_t parseLength(); - -private: - char const* _data; // start of the data blob - char const* _next; // start of the next message within the blob - - size_t _size; -}; - -/** - * The helper class encapsulating serialization operations - * with Google protobuf objects. - */ -class FrameBuffer { -public: - /// The default capacity of teh buffer - static const size_t DEFAULT_SIZE; - - /// Google protobuffers are more efficient below this size (bytes) - static const size_t DESIRED_LIMIT; - - /// The hard limit (bytes) for a single Google protobuffer - static const size_t HARD_LIMIT; - - /** - * Construct the buffer of the specified initial capacity (bytes). - */ - explicit FrameBuffer(size_t capacity = DEFAULT_SIZE); - - // Copy semantics are proxibited - - FrameBuffer(FrameBuffer const&) = delete; - FrameBuffer& operator=(FrameBuffer const&) = delete; - - /// Destructor - ~FrameBuffer(); - - /** - * @return pointer to the data blob - */ - char* data() { return _data; } - - /** - * @return maximum capacity (bytes) of the buffer - */ - size_t capacity() const { return _capacity; } - - /** - * @return meaninful size (bytes) of the buffer - */ - size_t size() const { return _size; } - - /** - * Set the size of the meaningful content of the buffer. If the buffer - * capacity is insufficient to accomodate the requested size the buffer - * will be extended. In the later case its previous content (if any) will - * be preserved. - * - * The method will throw one of these exceptions: - * - * std::overflow_error - * if the buffer doesn't have enough space to accomodate the request - * - * @param newSizeBytes - new size (bytes) of the buffer - */ - void resize(size_t newSizeBytes = 0); - - /** - * Add a message into the buffer. The message will be preceeed - * by a frame header carrying the length of the message. - * - * The method will throw one of these exceptions: - * - * std::overflow_error - * if the buffer doesn't have enough space to accomodate the data - * - * std::runtime_error - * if the serialization failed - * - * @param message - protobuf message to be serialized into the buffer - */ - template - void serialize(T const& message) { - uint32_t const headerLength = sizeof(uint32_t); - uint32_t const messageLength = message.ByteSizeLong(); - - // Make sure we have enough space to accomodate the frame header - // and the message body. - extend(_size + headerLength + messageLength); - - // Serialize the message header carrying the length of the message - *(reinterpret_cast(_data + _size)) = htonl(messageLength); - _size += headerLength; - - // Serialize the message itself - if (!message.SerializeToArray(_data + _size, _capacity - _size)) - throw FrameBufferError("FrameBuffer::serialize() ** message serialization failed **"); - - _size += messageLength; - } - -private: - /** - * Ensure the buffer capacity is no less than the specified number of bytes. - * Extend it otherwise. The previous contents (as per its 'size') of the buffer - * as well as its size will be preserved. - */ - void extend(size_t newCapacityBytes); - -private: - char* _data; // start of the allocated buffer - - size_t _capacity; - size_t _size; -}; - -} // namespace lsst::qserv::proto - -#endif // LSST_QSERV_PROTO_FRAME_BUFFER_H diff --git a/src/proto/ProtoHeaderWrap.cc b/src/proto/ProtoHeaderWrap.cc deleted file mode 100644 index 8b0d496856..0000000000 --- a/src/proto/ProtoHeaderWrap.cc +++ /dev/null @@ -1,45 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// System headers - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "proto/ProtoHeaderWrap.h" -#include "util/common.h" - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.parser.ProtoHeaderWrap"); -} - -namespace lsst::qserv::proto { - -// Google protobuffers are more efficient below 2MB, but xrootd is faster with larger limits. -// Reducing max to 2MB as it reduces the probablity of running out of memory. -const size_t ProtoHeaderWrap::PROTOBUFFER_DESIRED_LIMIT = 2000000; -// A single Google protobuffer can't be larger than this. -const size_t ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT = 64000000; - -} // namespace lsst::qserv::proto diff --git a/src/proto/ProtoHeaderWrap.h b/src/proto/ProtoHeaderWrap.h deleted file mode 100644 index d93624b7a2..0000000000 --- a/src/proto/ProtoHeaderWrap.h +++ /dev/null @@ -1,47 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2015-2016 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -#ifndef LSST_QSERV_PROTO_PROTO_HEADER_WRAP_H -#define LSST_QSERV_PROTO_PROTO_HEADER_WRAP_H -/** - * @file - * - * @brief Wrap the google protocol header in a fixed size container. - * - * @author John Gates, SLAC - */ - -// System headers -#include - -namespace lsst::qserv::proto { - -class ProtoHeaderWrap { -public: - static const size_t PROTOBUFFER_HARD_LIMIT; - static const size_t PROTOBUFFER_DESIRED_LIMIT; -}; - -} // namespace lsst::qserv::proto - -#endif diff --git a/src/proto/worker.proto b/src/proto/worker.proto deleted file mode 100644 index 1ccefaf5f9..0000000000 --- a/src/proto/worker.proto +++ /dev/null @@ -1,68 +0,0 @@ -/* - * LSST Data Management System - * Copyright 2011-2015 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// worker.proto -/// This defines the wire-messages sent between czar and worker. - -// After enabling this option, please visit Qserv code to allow -// Arena where it's protected by macro QSERV_USE_PROTO_ARENA. -syntax = "proto2"; -option cc_enable_arenas = true; - -package lsst.qserv.proto; - -// The file-based result delivery protocol has two kinds of messages. -// -// 1. The summary message sent back to Czar over the XROOTD/SSI protocol: -// -// - The length in bytes of the serialized ResponseSummary object (32-bits) -// - The serialized ResponseSummary object -// -// 2. The response data messages serialized and written into the result file -// -// - The length in bytes of the first serialized ResponseData object (32-bits) -// - The serialized first ResponseData object -// [ ... ] -// - The length in bytes of the last serialized ResponseData object (32-bits) -// - The serialized last ResponseData object - -message ResponseSummary { - required string wname = 1; - optional int32 errorcode = 2 [default = 0]; - optional string errormsg = 3 [default = ""]; - required uint64 queryid = 4; - required int32 jobid = 5; - optional uint32 rowcount = 6 [default = 0]; - optional uint64 transmitsize = 7 [default = 0]; - optional int32 attemptcount = 8 [default = 0]; - optional string fileresource_http = 10 [default = ""]; -} - -message RowBundle { - repeated bytes column = 1; // bytes to allow BLOB encoding - repeated bool isnull = 2; // Flag to allow sending nulls. -} - -message ResponseData { - repeated RowBundle row = 1; - required uint32 rowcount = 2; - required uint64 transmitsize = 3; -} diff --git a/src/qana/CMakeLists.txt b/src/qana/CMakeLists.txt index 30deb59caa..e254753a64 100644 --- a/src/qana/CMakeLists.txt +++ b/src/qana/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(qana SHARED) -add_dependencies(qana proto) target_sources(qana PRIVATE AggregatePlugin.cc diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index e19de24081..a44c174721 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(qdisp SHARED) -add_dependencies(qdisp proto) target_sources(qdisp PRIVATE ChunkMeta.cc diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index eea45f440b..5d9af5972e 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -33,7 +33,6 @@ #include "lsst/log/Log.h" // Qserv headers -#include "proto/worker.pb.h" #include "util/Bug.h" #include "qdisp/Executive.h" #include "qdisp/ResponseHandler.h" diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 276e680d9d..2713e40a95 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -32,19 +32,25 @@ // Qserv headers #include "util/Error.h" -// Forward declarations - -namespace lsst::qserv::proto { -class ResponseSummary; -} // namespace lsst::qserv::proto - -// This header declaration - namespace lsst::qserv::qdisp { class JobQuery; class UberJob; +/// Status of the merge at the end of merging. +/// contaminated can be true only if success is false. +class MergeEndStatus { +public: + MergeEndStatus() = default; + explicit MergeEndStatus(bool success_) : success(success_) {} + + /// True indicates the results were successfully merged + bool success = false; + + /// True indicates merge results are ruined and this query should be abandoned. + bool contaminated = false; +}; + /// ResponseHandler is an interface that handles result bytes. Tasks are /// submitted to an Executive instance naming a resource unit (what resource is /// required), a request string (task payload), and a handler for returning bytes. @@ -66,16 +72,12 @@ class ResponseHandler { virtual bool flush(proto::ResponseSummary const& responseSummary) = 0; /// Collect result data from the worker and merge it with the query result table. - /// If success, then everything is fine. - /// If not success, and not shouldCancel, the user query can be saved by abandoning - /// this UberJob. If shouldCancel is true, the result table is fouled and the user + /// If MergeEndStatus.success == true, then everything is fine. + /// If not .success, and not .contaminated, the user query can be saved by abandoning + /// this UberJob. If .contaminated is true, the result table is fouled and the user /// query is ruined. - /// @return success - true if the operation was successful - /// @return shouldCancel - if success was false, this being true indicates there - /// was an unrecoverable error in table writing and the query - /// should be cancelled. - virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t fileSize, - uint64_t expectedRows, uint64_t& resultRows) = 0; + /// @return - @see MergeEndStatus + virtual MergeEndStatus flushHttp(std::string const& fileUrl, uint64_t fileSize) = 0; /// Add the error to the error output if it is the first error. virtual void flushHttpError(int errorCode, std::string const& errorMsg, int status) = 0; diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 76391ee948..edfbcfce51 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -36,7 +36,6 @@ #include "global/LogContext.h" #include "http/Client.h" #include "http/MetaModule.h" -#include "proto/worker.pb.h" #include "protojson/UberJobMsg.h" #include "qdisp/CzarStats.h" #include "qdisp/JobQuery.h" @@ -309,35 +308,31 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); return; } - uint64_t resultRows = 0; - auto [flushSuccess, mergeHappened] = - ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize, rowCount, resultRows); + MergeEndStatus flushStatus = ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize); LOGS(_log, LOG_LVL_TRACE, - ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushSuccess - << " mergeHappened=" << mergeHappened); - if (flushSuccess) { + ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushStatus.success + << " contaminated=" << flushStatus.contaminated); + if (flushStatus.success) { qdisp::CzarStats::get()->addTotalRowsRecv(rowCount); qdisp::CzarStats::get()->addTotalBytesRecv(fileSize); } else { - bool flushShouldCancel = false; - if (mergeHappened) { + if (flushStatus.contaminated) { // This would probably indicate malformed file+rowCount or writing the result table failed. // If any merging happened, the result table is ruined. LOGS(_log, LOG_LVL_ERROR, ujPtr->cName(__func__) << "::fileCollectFunc flushHttp failed after merging, results ruined."); - flushShouldCancel = true; } else { // Perhaps something went wrong with file collection, so it is worth trying the jobs again // by abandoning this UberJob. LOGS(_log, LOG_LVL_ERROR, ujPtr->cName(__func__) << "::fileCollectFunc flushHttp failed, retrying Jobs."); } - ujPtr->_importResultError(flushShouldCancel, "mergeError", "merging failed"); + ujPtr->_importResultError(flushStatus.contaminated, "mergeError", "merging failed"); } // At this point all data for this job have been read. - ujPtr->_importResultFinish(resultRows); + ujPtr->_importResultFinish(rowCount); }; auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 277fb6dc82..9840e0b2f1 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -87,9 +87,8 @@ class TestInfo : public ResponseHandler { // virtual function that won't be needed - std::tuple flushHttp(std::string const& fileUrl, uint64_t fileSize, uint64_t expectedRows, - uint64_t& resultRows) override { - return {true, false}; + MergeEndStatus flushHttp(std::string const& fileUrl, uint64_t fileSize) override { + return MergeEndStatus(true); } void flushHttpError(int errorCode, std::string const& errorMsg, int status) override {} void errorFlush(std::string const& msg, int code) override {}; diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index 987a28594f..af9be1f5a6 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -28,7 +28,6 @@ add_executable(testQMeta testQMeta.cc) target_link_libraries(testQMeta cconfig - proto qmeta sql Boost::unit_test_framework diff --git a/src/qproc/CMakeLists.txt b/src/qproc/CMakeLists.txt index a8d7366273..34abafa571 100644 --- a/src/qproc/CMakeLists.txt +++ b/src/qproc/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(qproc SHARED) -add_dependencies(qproc proto) target_sources(qproc PRIVATE ChunkQuerySpec.cc diff --git a/src/query/CMakeLists.txt b/src/query/CMakeLists.txt index de326cb8fb..3c00fbe083 100644 --- a/src/query/CMakeLists.txt +++ b/src/query/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(query SHARED) -add_dependencies(query proto) target_sources(query PRIVATE AggOp.cc diff --git a/src/rproc/CMakeLists.txt b/src/rproc/CMakeLists.txt index 817a87e90a..304db90285 100644 --- a/src/rproc/CMakeLists.txt +++ b/src/rproc/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(rproc SHARED) -add_dependencies(rproc proto) target_sources(rproc PRIVATE InfileMerger.cc diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index b1d8275bf2..9b72bc9e69 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -57,7 +57,6 @@ #include "cconfig/CzarConfig.h" #include "global/intTypes.h" #include "mysql/CsvBuffer.h" -#include "proto/worker.pb.h" #include "qdisp/CzarStats.h" #include "qdisp/UberJob.h" #include "qproc/DatabaseModels.h" @@ -181,6 +180,25 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi return true; } + // Check if the final result size is too large. It should be safe to do this + // here as the only expected errors at this point are failures in transmission. + // Even if there is a failure in transmission, the retry would be expected + // to put the result size over the limit again. + { + lock_guard resultSzLock(_mtxResultSizeMtx); + _perJobResultSize[uJobId] += fileSize; + size_t tResultSize = _totalResultSize + _perJobResultSize[uJobId]; + if (tResultSize > _maxResultTableSizeBytes) { + ostringstream os; + os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable + << " is too large at " << tResultSize << " bytes, max allowed size is " + << _maxResultTableSizeBytes << " bytes"; + LOGS(_log, LOG_LVL_ERROR, os.str()); + _error = util::Error(-1, os.str(), -1); + return false; + } + } + TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, bool success) { if (!success) return; @@ -199,30 +217,9 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); virtFileT.stop(); - // &&& FOULED_RESULTS - // &&& At this point, it's probably possible to ask csvStream how many bytes were written. - // &&& If 0 bytes were written, the results should be ok and the query doesn't need to be cancelled. - - size_t const resultSize = fileSize; - size_t tResultSize; - { - std::lock_guard resultSzLock(_mtxResultSizeMtx); - _perJobResultSize[uJobId] += resultSize; - tResultSize = _totalResultSize + _perJobResultSize[uJobId]; - } - if (tResultSize > _maxResultTableSizeBytes) { - std::ostringstream os; - os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable - << " is too large at " << tResultSize << " bytes, max allowed size is " << _maxResultTableSizeBytes - << " bytes"; - LOGS(_log, LOG_LVL_ERROR, os.str()); - _error = util::Error(-1, os.str(), -1); - return false; - } - - tct->addToValue(resultSize); + tct->addToValue(fileSize); tct->setSuccess(); - tct.reset(); // stop transmit recieve timer before merging happens. + tct.reset(); // stop transmit receive timer before merging happens. // Stop here (if requested) after collecting stats on the amount of data collected // from workers. diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index 3334c74934..f86ea5db95 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -49,9 +49,7 @@ namespace mysql { class CsvStream; class MysqlConfig; } // namespace mysql -namespace proto { -class ResponseSummary; -} // namespace proto + namespace qdisp { class MessageStore; class UberJob; @@ -105,11 +103,6 @@ class InfileMerger { InfileMerger& operator=(InfileMerger const&) = delete; ~InfileMerger() = default; - /// Merge a worker response, which contains a single message - /// @return true if merge was successfully imported. - bool merge(proto::ResponseSummary const& responseSummary, - std::shared_ptr const& csvStream); - /// Merge the result data collected over Http. bool mergeHttp(std::shared_ptr const& uberJob, uint64_t fileSize, std::shared_ptr const& csvStream); diff --git a/src/sql/SqlResults.cc b/src/sql/SqlResults.cc index d7d918ec7b..36b89e085a 100644 --- a/src/sql/SqlResults.cc +++ b/src/sql/SqlResults.cc @@ -30,6 +30,9 @@ #include #include +// LSST headers +#include "lsst/log/Log.h" + // Qserv headers #include "mysql/SchemaFactory.h" @@ -38,6 +41,10 @@ // when the null pointer is passed into the constructor. #define EMPTY_STR_IF_NULL(x) ((x) == nullptr ? "" : (x)) +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.sql.SqlResults"); +} + namespace lsst::qserv::sql { namespace detail { @@ -117,12 +124,22 @@ void SqlResults::addResult(MYSQL_RES* r) { } } -bool SqlResults::extractFirstColumn(std::vector& ret, SqlErrorObject& errObj) { - int i, s = _results.size(); - for (i = 0; i < s; ++i) { +bool SqlResults::extractFirstXColumns(std::vector*> const& vectorRef, + SqlErrorObject& sqlErr) { + size_t rsz = _results.size(); + size_t expectedCols = vectorRef.size(); + if (rsz > 0 && mysql_num_fields(_results[0]) < expectedCols) { + LOGS(_log, LOG_LVL_ERROR, + "extractFirstXColumns had too few columns expected=" << rsz << " found=" + << mysql_num_fields(_results[0])); + return false; + } + for (size_t i = 0; i < rsz; ++i) { MYSQL_ROW row; while ((row = mysql_fetch_row(_results[i])) != nullptr) { - ret.push_back(EMPTY_STR_IF_NULL(row[0])); + for (size_t j = 0; j < expectedCols; ++j) { + vectorRef[j]->push_back(EMPTY_STR_IF_NULL(row[j])); + } } mysql_free_result(_results[i]); } @@ -130,74 +147,27 @@ bool SqlResults::extractFirstColumn(std::vector& ret, SqlErrorObjec return true; } +bool SqlResults::extractFirstColumn(std::vector& col1, SqlErrorObject& errObj) { + return extractFirstXColumns({&col1}, errObj); +} bool SqlResults::extractFirst2Columns(std::vector& col1, std::vector& col2, SqlErrorObject& errObj) { - int i, s = _results.size(); - for (i = 0; i < s; ++i) { - MYSQL_ROW row; - while ((row = mysql_fetch_row(_results[i])) != nullptr) { - col1.push_back(EMPTY_STR_IF_NULL(row[0])); - col2.push_back(EMPTY_STR_IF_NULL(row[1])); - } - mysql_free_result(_results[i]); - } - _results.clear(); - return true; + return extractFirstXColumns({&col1, &col2}, errObj); } - bool SqlResults::extractFirst3Columns(std::vector& col1, std::vector& col2, std::vector& col3, SqlErrorObject& errObj) { - int i, s = _results.size(); - for (i = 0; i < s; ++i) { - MYSQL_ROW row; - while ((row = mysql_fetch_row(_results[i])) != nullptr) { - col1.push_back(EMPTY_STR_IF_NULL(row[0])); - col2.push_back(EMPTY_STR_IF_NULL(row[1])); - col3.push_back(EMPTY_STR_IF_NULL(row[2])); - } - mysql_free_result(_results[i]); - } - _results.clear(); - return true; + return extractFirstXColumns({&col1, &col2, &col3}, errObj); } - bool SqlResults::extractFirst4Columns(std::vector& col1, std::vector& col2, std::vector& col3, std::vector& col4, SqlErrorObject& errObj) { - int i, s = _results.size(); - for (i = 0; i < s; ++i) { - MYSQL_ROW row; - while ((row = mysql_fetch_row(_results[i])) != nullptr) { - col1.push_back(EMPTY_STR_IF_NULL(row[0])); - col2.push_back(EMPTY_STR_IF_NULL(row[1])); - col3.push_back(EMPTY_STR_IF_NULL(row[2])); - col4.push_back(EMPTY_STR_IF_NULL(row[3])); - } - mysql_free_result(_results[i]); - } - _results.clear(); - return true; + return extractFirstXColumns({&col1, &col2, &col3, &col4}, errObj); } - bool SqlResults::extractFirst6Columns(std::vector& col1, std::vector& col2, std::vector& col3, std::vector& col4, std::vector& col5, std::vector& col6, SqlErrorObject& errObj) { - int i, s = _results.size(); - for (i = 0; i < s; ++i) { - MYSQL_ROW row; - while ((row = mysql_fetch_row(_results[i])) != nullptr) { - col1.push_back(EMPTY_STR_IF_NULL(row[0])); - col2.push_back(EMPTY_STR_IF_NULL(row[1])); - col3.push_back(EMPTY_STR_IF_NULL(row[2])); - col4.push_back(EMPTY_STR_IF_NULL(row[3])); - col5.push_back(EMPTY_STR_IF_NULL(row[4])); - col6.push_back(EMPTY_STR_IF_NULL(row[5])); - } - mysql_free_result(_results[i]); - } - _results.clear(); - return true; + return extractFirstXColumns({&col1, &col2, &col3, &col4, &col5, &col6}, errObj); } std::vector> SqlResults::extractFirstNColumns(size_t numColumns) { diff --git a/src/sql/SqlResults.h b/src/sql/SqlResults.h index f696b9185c..f5894b4c8d 100644 --- a/src/sql/SqlResults.h +++ b/src/sql/SqlResults.h @@ -90,17 +90,34 @@ class SqlResults : boost::noncopyable { // do not use it for SELECT unsigned long long getAffectedRows() const { return _affectedRows; } bool extractFirstValue(std::string&, SqlErrorObject&); - bool extractFirstColumn(std::vector&, SqlErrorObject&); - // TODO:UJ these extractFirst things are not pretty &&& - bool extractFirst2Columns(std::vector&, // FIXME: generalize - std::vector&, SqlErrorObject&); - bool extractFirst3Columns(std::vector&, // FIXME: generalize - std::vector&, std::vector&, SqlErrorObject&); - bool extractFirst4Columns(std::vector&, std::vector&, std::vector&, - std::vector&, SqlErrorObject&); - bool extractFirst6Columns(std::vector&, std::vector&, std::vector&, - std::vector&, std::vector&, std::vector&, - SqlErrorObject&); + + /// Return the value of the first X columns of `_results`, where X is the size() of vectorRef. + /// It would be nice to use references instead of pointers, but curly bracket initialization + /// of the references was problematic. + /// @param vectorRef - A vector of pointers to vectors of strings. Each vector of strings + /// contains a column of the table (index 0 holds column1, + /// index 1 holds column2, etc.). The number of columns returned is + /// vectorRef.size(). NULL values are set to empty strings. + /// @param errObj - is never set and should be removed. (Only likely error is database disconnect, + /// which would be catastrophic) + /// @return - Returns false when fewer than expected columns are found. + // TODO:UJ for most of these functions, calling extractFirstXColumns + // directly may make more sense than calling extractFirst6Columns. + // Not changing this now as it will make rebasing difficult. + bool extractFirstXColumns(std::vector*> const& vectorRef, + SqlErrorObject& sqlErr); + bool extractFirstColumn(std::vector& col1, SqlErrorObject& errObj); + bool extractFirst2Columns(std::vector& col1, std::vector& col2, + SqlErrorObject& errObj); + bool extractFirst3Columns(std::vector& col1, std::vector& col2, + std::vector& col3, SqlErrorObject& errObj); + bool extractFirst4Columns(std::vector& col1, std::vector& col2, + std::vector& col3, std::vector& col4, + SqlErrorObject& errObj); + bool extractFirst6Columns(std::vector& col1, std::vector& col2, + std::vector& col3, std::vector& col4, + std::vector& col5, std::vector& col6, + SqlErrorObject& errObj); template bool extractFirstColumns(SqlErrorObject& err, Columns&... cols) { diff --git a/src/util/ResultFileName.cc b/src/util/ResultFileName.cc index 300e682198..acd4097400 100644 --- a/src/util/ResultFileName.cc +++ b/src/util/ResultFileName.cc @@ -25,6 +25,9 @@ // Third party headers #include "boost/filesystem.hpp" +// LSST headers +#include "lsst/log/Log.h" + // Qserv headers #include "util/String.h" @@ -32,19 +35,28 @@ using namespace std; using namespace nlohmann; namespace fs = boost::filesystem; +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.util.ResultFileName"); +} + namespace lsst::qserv::util { string const ResultFileName::fileExt = ".csv"; -ResultFileName::ResultFileName(qmeta::CzarId czarId, QueryId queryId, uint32_t jobId, uint32_t chunkId, - uint32_t attemptCount) - : _fileName(to_string(czarId) + "-" + to_string(queryId) + "-" + to_string(jobId) + "-" + - to_string(chunkId) + "-" + to_string(attemptCount) + fileExt), +ResultFileName::ResultFileName(qmeta::CzarId czarId, QueryId queryId, UberJobId ujId) + : _fileName(to_string(czarId) + splitToken() + to_string(queryId) + splitToken() + to_string(ujId) + + fileExt), _czarId(czarId), _queryId(queryId), - _jobId(jobId), - _chunkId(chunkId), - _attemptCount(attemptCount) {} + _ujId(ujId) { + if (ujId < 0) { + string const eMsg = + "ResultFileName cannot be called with negative, which would be invalid for a filename " + + _fileName; + LOGS(_log, LOG_LVL_ERROR, eMsg); + throw std::invalid_argument(eMsg); + } +} ResultFileName::ResultFileName(fs::path const& filePath) : _fileName(filePath.filename().string()) { _parse(); @@ -55,11 +67,7 @@ ResultFileName::ResultFileName(string const& filePath) : _fileName(fs::path(file } json ResultFileName::toJson() const { - return json::object({{"czar_id", _czarId}, - {"query_id", _queryId}, - {"job_id", _jobId}, - {"chunk_id", _chunkId}, - {"attemptcount", _attemptCount}}); + return json::object({{"czar_id", _czarId}, {"query_id", _queryId}, {"uj_id", _ujId}}); } bool ResultFileName::operator==(ResultFileName const& rhs) const { return _fileName == rhs._fileName; } @@ -78,16 +86,14 @@ void ResultFileName::_parse() { throw invalid_argument(_context(__func__) + " not a valid result file name: " + _fileName + ", file ext: " + fileNameExt + ", expected: " + fileExt); } - _taskAttributes = String::parseToVectUInt64(fileName.stem().string(), "-"); - if (_taskAttributes.size() != 5) { + _taskAttributes = String::parseToVectUInt64(fileName.stem().string(), splitToken()); + if (_taskAttributes.size() != 3) { throw invalid_argument(_context(__func__) + " not a valid result file name: " + _fileName); } size_t attrIndex = 0; _validateAndStoreAttr(attrIndex++, "czarId", _czarId); _validateAndStoreAttr(attrIndex++, "queryId", _queryId); - _validateAndStoreAttr(attrIndex++, "jobId", _jobId); - _validateAndStoreAttr(attrIndex++, "chunkId", _chunkId); - _validateAndStoreAttr(attrIndex++, "attemptCount", _attemptCount); + _validateAndStoreAttr(attrIndex++, "ujId", _ujId); } } // namespace lsst::qserv::util diff --git a/src/util/ResultFileName.h b/src/util/ResultFileName.h index 424f2fa2cb..2b5498cdee 100644 --- a/src/util/ResultFileName.h +++ b/src/util/ResultFileName.h @@ -58,9 +58,12 @@ namespace lsst::qserv::util { * All operations are done in the class's constructors. A few forms of the construction are * provided for convenience of the client applications. * + * QueryId + UberJobId results is a unique identifier. + * CzarId can be useful for some operations. + * * The file path has the following general format: * @code - * [/]----[.] + * [/]-[.] * @code */ class ResultFileName { @@ -68,14 +71,15 @@ class ResultFileName { /// The file extention including the '.' prefix. static std::string const fileExt; + static std::string splitToken() { return std::string("-"); } + ResultFileName() = default; ResultFileName(ResultFileName const&) = default; ResultFileName& operator=(ResultFileName const&) = default; - /// This form of constructionstores attributes of a file and generates + /// This form of construction stores attributes of a file and generates /// the name of a file in a format specified in the class description section. - ResultFileName(qmeta::CzarId czarId, QueryId queryId, std::uint32_t jobId, std::uint32_t chunkId, - std::uint32_t attemptCount); + ResultFileName(qmeta::CzarId czarId, QueryId queryId, UberJobId ujId); /// @param filePath The file to be evaluated. /// @throw std::invalid_argument If the file path did not match expectations. @@ -90,9 +94,7 @@ class ResultFileName { qmeta::CzarId czarId() const { return _czarId; } QueryId queryId() const { return _queryId; } - std::uint32_t jobId() const { return _jobId; } - std::uint32_t chunkId() const { return _chunkId; } - std::uint32_t attemptCount() const { return _attemptCount; } + UberJobId ujId() const { return _ujId; } /// @return The JSON object (dictionary) encapsulating values of the attributes. nlohmann::json toJson() const; @@ -106,26 +108,26 @@ class ResultFileName { static std::string _context(std::string const& func); void _parse(); + // This only works with unsigned, which wouldn't work with UberJobId + // except that negative UberJobId's never make it off of the czar. template void _validateAndStoreAttr(std::size_t attrIndex, std::string const& attrName, T& attr) { - std::uint64_t const& attrValue = _taskAttributes[attrIndex]; - T const minVal = std::numeric_limits::min(); - T const maxVal = std::numeric_limits::max(); - if ((attrValue >= minVal) && (attrValue <= maxVal)) { + size_t const& attrValue = _taskAttributes[attrIndex]; + size_t const maxVal = std::numeric_limits::max(); + /// min value for size_t is 0, so only max matters + if (attrValue <= maxVal) { attr = static_cast(attrValue); return; } - throw std::invalid_argument(_context(__func__) + " failed for attribute=" + attrName + ", value=" + - std::to_string(attrValue) + ", allowed range=[" + std::to_string(minVal) + - "," + std::to_string(maxVal) + "], file=" + _fileName); + throw std::invalid_argument(_context(__func__) + " failed for attribute=" + attrName + + ", value=" + std::to_string(attrValue) + ", allowed range=[0," + + std::to_string(maxVal) + "], file=" + _fileName); } std::string _fileName; qmeta::CzarId _czarId = 0; QueryId _queryId = 0; - std::uint32_t _jobId = 0; - std::uint32_t _chunkId = 0; - std::uint32_t _attemptCount = 0; + UberJobId _ujId = 0; std::vector _taskAttributes; }; diff --git a/src/util/testResultFileName.cc b/src/util/testResultFileName.cc index cdb978f05f..f8ba13e955 100644 --- a/src/util/testResultFileName.cc +++ b/src/util/testResultFileName.cc @@ -51,22 +51,17 @@ BOOST_AUTO_TEST_CASE(ResultFileNameTest) { lsst::qserv::qmeta::CzarId const czarId = 1; lsst::qserv::QueryId const queryId = 2; - uint32_t const jobId = 3; - uint32_t const chunkId = 4; - uint32_t const attemptCount = 5; + lsst::qserv::UberJobId const ujId = 3; std::string const name2parse = std::to_string(czarId) + "-" + std::to_string(queryId) + "-" + - std::to_string(jobId) + "-" + std::to_string(chunkId) + "-" + - std::to_string(attemptCount) + lsst::qserv::util::ResultFileName::fileExt; + std::to_string(ujId) + lsst::qserv::util::ResultFileName::fileExt; BOOST_CHECK_NO_THROW({ lsst::qserv::util::ResultFileName const file(name2parse); BOOST_CHECK_EQUAL(file.fileName(), name2parse); BOOST_CHECK_EQUAL(file.czarId(), czarId); BOOST_CHECK_EQUAL(file.queryId(), queryId); - BOOST_CHECK_EQUAL(file.jobId(), jobId); - BOOST_CHECK_EQUAL(file.chunkId(), chunkId); - BOOST_CHECK_EQUAL(file.attemptCount(), attemptCount); + BOOST_CHECK_EQUAL(file.ujId(), ujId); }); BOOST_CHECK_NO_THROW({ @@ -74,9 +69,7 @@ BOOST_AUTO_TEST_CASE(ResultFileNameTest) { BOOST_CHECK_EQUAL(file.fileName(), name2parse); BOOST_CHECK_EQUAL(file.czarId(), czarId); BOOST_CHECK_EQUAL(file.queryId(), queryId); - BOOST_CHECK_EQUAL(file.jobId(), jobId); - BOOST_CHECK_EQUAL(file.chunkId(), chunkId); - BOOST_CHECK_EQUAL(file.attemptCount(), attemptCount); + BOOST_CHECK_EQUAL(file.ujId(), ujId); }); BOOST_CHECK_NO_THROW({ @@ -84,27 +77,22 @@ BOOST_AUTO_TEST_CASE(ResultFileNameTest) { BOOST_CHECK_EQUAL(file.fileName(), name2parse); BOOST_CHECK_EQUAL(file.czarId(), czarId); BOOST_CHECK_EQUAL(file.queryId(), queryId); - BOOST_CHECK_EQUAL(file.jobId(), jobId); - BOOST_CHECK_EQUAL(file.chunkId(), chunkId); - BOOST_CHECK_EQUAL(file.attemptCount(), attemptCount); + BOOST_CHECK_EQUAL(file.ujId(), ujId); }); BOOST_CHECK_NO_THROW({ - lsst::qserv::util::ResultFileName const file(czarId, queryId, jobId, chunkId, attemptCount); + lsst::qserv::util::ResultFileName const file(czarId, queryId, ujId); BOOST_CHECK_EQUAL(file.fileName(), name2parse); BOOST_CHECK_EQUAL(file.czarId(), czarId); BOOST_CHECK_EQUAL(file.queryId(), queryId); - BOOST_CHECK_EQUAL(file.jobId(), jobId); - BOOST_CHECK_EQUAL(file.chunkId(), chunkId); - BOOST_CHECK_EQUAL(file.attemptCount(), attemptCount); + BOOST_CHECK_EQUAL(file.ujId(), ujId); }); BOOST_CHECK_THROW( - { lsst::qserv::util::ResultFileName const file(std::string("1-2-3-4")); }, std::invalid_argument); + { lsst::qserv::util::ResultFileName const file(std::string("1-2")); }, std::invalid_argument); BOOST_CHECK_THROW( - { lsst::qserv::util::ResultFileName const file(std::string("a-2-3-4-5")); }, - std::invalid_argument); + { lsst::qserv::util::ResultFileName const file(std::string("a-2-3-4")); }, std::invalid_argument); } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wbase/Base.h b/src/wbase/Base.h index bc80273804..8dc1b5662a 100644 --- a/src/wbase/Base.h +++ b/src/wbase/Base.h @@ -37,15 +37,10 @@ class XrdSysError; class XrdSysLogger; class XrdSfsAio; -namespace lsst::qserv { -namespace proto { -class TaskMsg; -} -namespace wbase { +namespace lsst::qserv { namespace wbase { class StringBuffer; class StringBuffer2; -} // namespace wbase -} // namespace lsst::qserv +}} // namespace lsst::qserv::wbase namespace lsst::qserv::wbase { diff --git a/src/wbase/CMakeLists.txt b/src/wbase/CMakeLists.txt index 1de19abbf5..4a420fb9fe 100644 --- a/src/wbase/CMakeLists.txt +++ b/src/wbase/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(wbase SHARED) -add_dependencies(wbase proto) target_sources(wbase PRIVATE Base.cc diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index d3c76b88e4..54a61ba07a 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -35,8 +35,6 @@ // Qserv headers #include "global/LogContext.h" #include "mysql/MySqlUtils.h" -#include "proto/ProtoHeaderWrap.h" -#include "proto/worker.pb.h" #include "wbase/Task.h" #include "wbase/UberJobData.h" #include "wconfig/WorkerConfig.h" diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 57ab7d2dbd..f0ae325c28 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -50,7 +50,7 @@ class MultiError; namespace lsst::qserv::wbase { class UberJobData; -/// The class is responsible for writing mysql result rows as Protobuf +/// The class is responsible for writing mysql result rows as Csv /// serialized messages into an output file. Once a task (or all sub-chunk /// tasks) finished writing data a short reply message is sent back to Czar using /// SSI request's SendChannel that was provided to the factory method @@ -181,9 +181,9 @@ class FileChannelShared { * Transfer rows of the result set into into the output file. * @note The file will be created at the first call to the method. * @note The method may not extract all rows if the amount of data found - * in the result set exceeded the maximum size allowed by the Google Protobuf - * implementation. Also, the iterative approach to the data extraction allows - * the driving code to be interrupted should the correponding query be cancelled + * in the result set exceeded the maximum size allowed. Also, the iterative + * approach to the data extraction allows the driving code to be + * interrupted should the corresponding query be cancelled * during the lengthy data processing phase. * @param tMtxLock - a lock on the mutex tMtx * @param task - a task that produced the result set diff --git a/src/wbase/SendChannel.cc b/src/wbase/SendChannel.cc index c07dd37f63..f8a5485cda 100644 --- a/src/wbase/SendChannel.cc +++ b/src/wbase/SendChannel.cc @@ -37,7 +37,6 @@ #include "lsst/log/Log.h" // Qserv headers -#include "proto/ProtoHeaderWrap.h" #include "global/LogContext.h" #include "util/common.h" #include "util/Timer.h" diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 5beff3c5a0..24e4e65908 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -47,7 +47,6 @@ #include "global/UnsupportedError.h" #include "http/RequestBodyJSON.h" #include "mysql/MySqlConfig.h" -#include "proto/worker.pb.h" #include "protojson/UberJobMsg.h" #include "util/Bug.h" #include "util/common.h" diff --git a/src/wbase/TaskState.h b/src/wbase/TaskState.h index 5a2f92e61c..76063a780c 100644 --- a/src/wbase/TaskState.h +++ b/src/wbase/TaskState.h @@ -38,10 +38,6 @@ namespace lsst::qserv::wbase { * @note This class and the relevant functions are put into this header to * allow the complile-time (only) dependency onto this type from other modules * without needing to link against the current module's library. - * Also note a choice of the underlying type which is meant to allow sending - * values of the type as numeric attribites in the Protobuf messages w/o - * introducing an additional (Protobuf) representation for those, or converting - * the values to strings and vs. */ enum class TaskState : std::uint64_t { CREATED = 0, diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 0a805ae43c..289ea72e6c 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -42,6 +42,7 @@ #include "protojson/UberJobReadyMsg.h" #include "util/Bug.h" #include "util/MultiError.h" +#include "util/ResultFileName.h" #include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" #include "wpublish/ChunkInventory.h" @@ -192,11 +193,9 @@ void UberJobData::_queueUJResponse(http::Method method_, std::vectorgetFqdn() + ":" + to_string(_resultsHttpPort) + "/" + _resultFileName(); } diff --git a/src/wconfig/CMakeLists.txt b/src/wconfig/CMakeLists.txt index bc667a7ea1..95833d08d5 100644 --- a/src/wconfig/CMakeLists.txt +++ b/src/wconfig/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(wconfig SHARED) -add_dependencies(wconfig proto) target_sources(wconfig PRIVATE WorkerConfig.cc @@ -7,7 +6,6 @@ target_sources(wconfig PRIVATE target_link_libraries(wconfig PUBLIC log - proto ) install( diff --git a/src/wconfig/WorkerConfig.h b/src/wconfig/WorkerConfig.h index e4896c5a00..6b0736bfff 100644 --- a/src/wconfig/WorkerConfig.h +++ b/src/wconfig/WorkerConfig.h @@ -48,57 +48,6 @@ class AuthContext; // This header declarations namespace lsst::qserv::wconfig { -/// This class handles the special case for the configuration value representing -/// the communications protocol used which can have a text value of "HTTP" or " -/// "XROOTD", case-insenitive. -class ConfigValResultDeliveryProtocol : public util::ConfigVal { -public: - using CvrdpPtr = std::shared_ptr; - enum TEnum { - HTTP = 0, ///< Use HTTP protocol - XROOT = 1 ///< Use XROOTD file protocol - }; - - ConfigValResultDeliveryProtocol() = delete; - virtual ~ConfigValResultDeliveryProtocol() = default; - - static CvrdpPtr create(util::ConfigValMap& configValMap, std::string const& section, - std::string const& name, bool required, std::string const& defVal, - bool hidden = false) { - auto newPtr = CvrdpPtr(new ConfigValResultDeliveryProtocol(section, name, required, defVal, hidden)); - addToMapBase(configValMap, newPtr); - return newPtr; - } - - /// Return the appropriate TEnum for the given `str`, where "" returns HTTP. - /// @throws ConfigException - static TEnum parse(std::string const& str); - - /// Convert the TEnum `protocol` to the appropriate string. - static std::string toString(TEnum protocol); - - /// Return the string value of this object. - std::string getValStrDanger() const override { return toString(_val); } - - /// Return the string default value of this object. - std::string getDefValStrDanger() const override { return toString(_defVal); } - - void setValFromConfigStoreChild(util::ConfigStore const& configStore) override; - TEnum getVal() const { return _val; } - - void setVal(TEnum val) { - _val = val; - logValSet(); - } - -private: - ConfigValResultDeliveryProtocol(std::string const& section, std::string const& name, bool required, - std::string const& defVal, bool hidden) - : ConfigVal(section, name, required, hidden), _defVal(parse(defVal)), _val(_defVal) {} - TEnum const _defVal; ///< Default value for the item this class is storing. - TEnum _val; ///< Value for the item this class is storing. -}; - /// Provide all configuration parameters for a Qserv worker instance. /// Parse an INI configuration file, identify required parameters and ignore /// others, analyze and store them inside private member variables, use default diff --git a/src/wcontrol/CMakeLists.txt b/src/wcontrol/CMakeLists.txt index 9f447ed0a2..9dd1f453a1 100644 --- a/src/wcontrol/CMakeLists.txt +++ b/src/wcontrol/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(wcontrol SHARED) -add_dependencies(wcontrol proto) target_sources(wcontrol PRIVATE Foreman.cc diff --git a/src/wdb/CMakeLists.txt b/src/wdb/CMakeLists.txt index 21546e1daa..84a379a78a 100644 --- a/src/wdb/CMakeLists.txt +++ b/src/wdb/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(wdb SHARED) -add_dependencies(wdb proto) target_sources(wdb PRIVATE ChunkResource.cc diff --git a/src/wdb/ChunkResource.h b/src/wdb/ChunkResource.h index f030c23f37..68b83b0238 100644 --- a/src/wdb/ChunkResource.h +++ b/src/wdb/ChunkResource.h @@ -50,14 +50,9 @@ #include "wdb/SQLBackend.h" // Forward declarations -namespace lsst::qserv { -namespace proto { -class TaskMsg_Fragment; -} -namespace wdb { +namespace lsst::qserv { namespace wdb { class Task; -} -} // namespace lsst::qserv +}} // namespace lsst::qserv::wdb namespace lsst::qserv::wdb { diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index 6ceb1af217..3e2aa18d55 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -38,7 +38,6 @@ #include // Third-party headers -#include #include // Class header @@ -52,8 +51,6 @@ #include "mysql/MySqlConfig.h" #include "mysql/MySqlConnection.h" #include "mysql/SchemaFactory.h" -#include "proto/ProtoHeaderWrap.h" -#include "proto/worker.pb.h" #include "sql/Schema.h" #include "sql/SqlErrorObject.h" #include "util/Bug.h" diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc index 4e4bb36a08..17731d05f9 100644 --- a/src/wdb/testQueryRunner.cc +++ b/src/wdb/testQueryRunner.cc @@ -30,7 +30,6 @@ // Qserv headers #include "mysql/MySqlConfig.h" #include "protojson/ScanTableInfo.h" -#include "proto/worker.pb.h" #include "wbase/FileChannelShared.h" #include "wbase/Task.h" #include "wbase/UberJobData.h" @@ -48,7 +47,6 @@ using namespace std; namespace test = boost::test_tools; -namespace gio = google::protobuf::io; namespace util = lsst::qserv::util; using lsst::qserv::mysql::MySqlConfig; diff --git a/src/wpublish/CMakeLists.txt b/src/wpublish/CMakeLists.txt index 946b0c665a..d756bcad19 100644 --- a/src/wpublish/CMakeLists.txt +++ b/src/wpublish/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(wpublish SHARED) -add_dependencies(wpublish proto) target_sources(wpublish PRIVATE ChunkInventory.cc @@ -13,7 +12,6 @@ install( target_link_libraries(wpublish PUBLIC log - proto protobuf ) diff --git a/src/wsched/BlendScheduler.cc b/src/wsched/BlendScheduler.cc index 0d1ab2c0a9..9fd8a883b3 100644 --- a/src/wsched/BlendScheduler.cc +++ b/src/wsched/BlendScheduler.cc @@ -44,7 +44,6 @@ // Qserv headers #include "global/LogContext.h" -#include "proto/worker.pb.h" #include "util/Bug.h" #include "util/EventThread.h" #include "util/Timer.h" diff --git a/src/wsched/CMakeLists.txt b/src/wsched/CMakeLists.txt index e3cab0d3f6..0f4c2ea434 100644 --- a/src/wsched/CMakeLists.txt +++ b/src/wsched/CMakeLists.txt @@ -1,5 +1,4 @@ add_library(wsched SHARED) -add_dependencies(wsched proto) target_sources(wsched PRIVATE BlendScheduler.cc diff --git a/src/wsched/GroupScheduler.cc b/src/wsched/GroupScheduler.cc index 2429f7ee3b..904bcb3186 100644 --- a/src/wsched/GroupScheduler.cc +++ b/src/wsched/GroupScheduler.cc @@ -42,7 +42,6 @@ // Qserv headers #include "global/LogContext.h" -#include "proto/worker.pb.h" namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.wsched.GroupScheduler"); diff --git a/src/wsched/testSchedulers.cc b/src/wsched/testSchedulers.cc index f2217b1b68..7375991814 100644 --- a/src/wsched/testSchedulers.cc +++ b/src/wsched/testSchedulers.cc @@ -32,7 +32,6 @@ // Qserv headers #include "mysql/MySqlConfig.h" -#include "proto/worker.pb.h" #include "protojson/ScanTableInfo.h" #include "util/Command.h" #include "util/EventThread.h" @@ -60,7 +59,6 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wsched.testSchedulers"); using namespace std; using lsst::qserv::mysql::MySqlConfig; -using lsst::qserv::proto::TaskMsg; using lsst::qserv::wbase::FileChannelShared; using lsst::qserv::wbase::SendChannel; using lsst::qserv::wbase::Task; @@ -96,8 +94,6 @@ Task::Ptr makeTask(std::shared_ptr tm, shared_ptr con */ struct SchedulerFixture { - typedef std::shared_ptr TaskMsgPtr; - SchedulerFixture(void) { counter = 20; } ~SchedulerFixture(void) {} From 816ff5b8ccabbd40f4dc6c7f91922bb7e4c8e016 Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 14 May 2025 16:45:14 -0700 Subject: [PATCH 12/15] Added worker executable. --- .github/workflows/ci.yml | 24 +- deploy/compose/docker-compose.yml | 145 ++---------- deploy/compose/log/log-czar-proxy.cnf | 2 - ...g-worker-xrootd.cnf => log-worker-svc.cnf} | 1 - deploy/compose/log/log.cnf | 3 - deploy/docker/base/Dockerfile | 24 +- etc/log.cnf | 1 - python/lsst/qserv/admin/cli/entrypoint.py | 194 ++--------------- python/lsst/qserv/admin/cli/options.py | 15 -- python/lsst/qserv/admin/cli/script.py | 146 +------------ src/CMakeLists.txt | 6 +- .../templates/http/etc/qserv-czar.cnf.jinja | 21 -- .../templates/proxy/etc/qserv-czar.cnf.jinja | 21 -- .../etc/worker-svc.cf.jinja} | 6 +- .../xrootd/etc/cmsd-manager.cf.jinja | 49 ----- .../templates/xrootd/etc/cmsd-worker.cf.jinja | 68 ------ .../xrootd/etc/xrootd-manager.cf.jinja | 43 ---- src/ccontrol/MergingHandler.cc | 42 ++-- src/ccontrol/MergingHandler.h | 23 +- src/ccontrol/UserQuerySelect.cc | 16 +- src/czar/Czar.h | 4 +- src/czar/CzarChunkMap.cc | 32 ++- src/czar/CzarChunkMap.h | 6 +- src/czar/HttpCzarWorkerModule.cc | 7 +- src/mysql/CMakeLists.txt | 2 + src/mysql/CsvBuffer.cc | 27 ++- src/mysql/CsvBuffer.h | 19 ++ src/qdisp/CzarStats.cc | 12 +- src/qdisp/CzarStats.h | 4 +- src/qdisp/Executive.cc | 10 + src/qdisp/Executive.h | 5 +- src/qdisp/JobQuery.cc | 4 +- src/qdisp/ResponseHandler.h | 3 + src/qdisp/UberJob.cc | 60 +++-- src/qdisp/UberJob.h | 5 +- src/qdisp/testQDisp.cc | 1 + src/qhttp/Server.cc | 7 +- src/qmeta/JobStatus.h | 2 +- src/qmeta/MessageStore.cc | 20 +- src/qmeta/MessageStore.h | 5 + src/qmeta/QMetaMysql.cc | 8 +- src/replica/CMakeLists.txt | 2 - src/replica/apps/CMakeLists.txt | 2 - src/replica/apps/QservXrootdSsiApp.cc | 206 ------------------ src/replica/apps/QservXrootdSsiApp.h | 74 ------- src/replica/qserv/CMakeLists.txt | 1 - src/replica/qserv/QservMgtServices.h | 2 - src/replica/qserv/XrdCmsgetVnId.cc | 110 ---------- src/replica/tools/qserv-replica-test.cc | 2 - src/rproc/InfileMerger.cc | 29 ++- src/rproc/InfileMerger.h | 1 - src/sql/SqlResults.cc | 3 + src/sql/SqlResults.h | 3 + src/util/ConfigStore.h | 2 +- src/util/Error.h | 4 +- src/util/FileMonitor.cc | 5 +- src/util/QdispPool.h | 6 +- src/wbase/Base.h | 7 +- src/wbase/FileChannelShared.h | 2 +- src/wbase/SendChannel.cc | 2 +- src/wbase/SendChannel.h | 6 +- src/wbase/Task.h | 1 - src/wbase/UberJobData.cc | 3 +- src/wcomms/CMakeLists.txt | 18 ++ src/{xrdsvc => wcomms}/HttpModule.cc | 6 +- src/{xrdsvc => wcomms}/HttpModule.h | 10 +- src/{xrdsvc => wcomms}/HttpMonitorModule.cc | 6 +- src/{xrdsvc => wcomms}/HttpMonitorModule.h | 14 +- .../HttpReplicaMgtModule.cc | 26 +-- src/{xrdsvc => wcomms}/HttpReplicaMgtModule.h | 20 +- src/{xrdsvc => wcomms}/HttpSvc.cc | 18 +- src/{xrdsvc => wcomms}/HttpSvc.h | 12 +- .../HttpWorkerCzarModule.cc | 10 +- src/{xrdsvc => wcomms}/HttpWorkerCzarModule.h | 18 +- src/wconfig/WorkerConfig.h | 5 - src/wcontrol/Foreman.cc | 2 +- src/wcontrol/Foreman.h | 1 + src/wdb/CMakeLists.txt | 13 +- src/wdb/ChunkResource.h | 4 +- src/wdb/QueryRunner.cc | 3 +- src/wdb/QueryRunner.h | 3 +- src/wdb/SQLBackend.cc | 4 +- src/wmain/CMakeLists.txt | 47 ++++ .../SsiService.cc => wmain/WorkerMain.cc} | 71 +++--- .../SsiService.h => wmain/WorkerMain.h} | 66 +++--- src/wmain/qserv-worker-http.cc | 82 +++++++ src/wpublish/CMakeLists.txt | 13 +- src/wpublish/ChunkInventory.cc | 2 - src/wsched/CMakeLists.txt | 12 +- src/xrdlog/CMakeLists.txt | 18 -- src/xrdlog/XrdLogging.cc | 72 ------ src/xrdsvc/CMakeLists.txt | 37 ---- src/xrdsvc/SsiProvider.cc | 162 -------------- src/xrdsvc/SsiProvider.h | 79 ------- src/xrdsvc/XrdName.h | 61 ------ 95 files changed, 653 insertions(+), 1828 deletions(-) rename deploy/compose/log/{log-worker-xrootd.cnf => log-worker-svc.cnf} (87%) rename src/admin/templates/{xrootd/etc/xrdssi.cf.jinja => worker-svc/etc/worker-svc.cf.jinja} (95%) delete mode 100644 src/admin/templates/xrootd/etc/cmsd-manager.cf.jinja delete mode 100644 src/admin/templates/xrootd/etc/cmsd-worker.cf.jinja delete mode 100644 src/admin/templates/xrootd/etc/xrootd-manager.cf.jinja delete mode 100644 src/replica/apps/QservXrootdSsiApp.cc delete mode 100644 src/replica/apps/QservXrootdSsiApp.h delete mode 100644 src/replica/qserv/XrdCmsgetVnId.cc create mode 100644 src/wcomms/CMakeLists.txt rename src/{xrdsvc => wcomms}/HttpModule.cc (97%) rename src/{xrdsvc => wcomms}/HttpModule.h (93%) rename src/{xrdsvc => wcomms}/HttpMonitorModule.cc (97%) rename src/{xrdsvc => wcomms}/HttpMonitorModule.h (91%) rename src/{xrdsvc => wcomms}/HttpReplicaMgtModule.cc (94%) rename src/{xrdsvc => wcomms}/HttpReplicaMgtModule.h (94%) rename src/{xrdsvc => wcomms}/HttpSvc.cc (95%) rename src/{xrdsvc => wcomms}/HttpSvc.h (95%) rename src/{xrdsvc => wcomms}/HttpWorkerCzarModule.cc (98%) rename src/{xrdsvc => wcomms}/HttpWorkerCzarModule.h (90%) create mode 100644 src/wmain/CMakeLists.txt rename src/{xrdsvc/SsiService.cc => wmain/WorkerMain.cc} (86%) rename src/{xrdsvc/SsiService.h => wmain/WorkerMain.h} (53%) create mode 100644 src/wmain/qserv-worker-http.cc delete mode 100644 src/xrdlog/CMakeLists.txt delete mode 100644 src/xrdlog/XrdLogging.cc delete mode 100644 src/xrdsvc/CMakeLists.txt delete mode 100644 src/xrdsvc/SsiProvider.cc delete mode 100644 src/xrdsvc/SsiProvider.h delete mode 100644 src/xrdsvc/XrdName.h diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e69064be8d..563cd21aa2 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -305,37 +305,21 @@ jobs: if: always() run: docker logs ${USER}-czar-http-1 - - name: Czar CMSD Log - if: always() - run: docker logs ${USER}-czar-cmsd-1 - - - name: Czar XROOTD Log - if: always() - run: docker logs ${USER}-czar-xrootd-1 - - name: Czar MariaDB Log if: always() run: docker logs ${USER}-czar-mariadb-1 - - name: Qzerv Worker 0 CMSD Log + - name: Qzerv Worker 0 worker-svc Log if: always() - run: docker logs ${USER}-worker-cmsd-0-1 - - - name: Qzerv Worker 0 XROOTD Log - if: always() - run: docker logs ${USER}-worker-xrootd-0-1 + run: docker logs ${USER}-worker-svc-0-1 - name: Qzerv Worker 0 MariaDB Log if: always() run: docker logs ${USER}-worker-mariadb-0-1 - - name: Qzerv Worker 1 CMSD Log - if: always() - run: docker logs ${USER}-worker-cmsd-1-1 - - - name: Qzerv Worker 1 XROOTD Log + - name: Qzerv Worker 1 worker-svc Log if: always() - run: docker logs ${USER}-worker-xrootd-1-1 + run: docker logs ${USER}-worker-svc-1-1 - name: Qzerv Worker 1 MariaDB Log if: always() diff --git a/deploy/compose/docker-compose.yml b/deploy/compose/docker-compose.yml index d1ea8ec323..ff7245c4d8 100644 --- a/deploy/compose/docker-compose.yml +++ b/deploy/compose/docker-compose.yml @@ -15,18 +15,11 @@ x-log-volume: - type: bind source: ./log/ target: /config-etc/log/ -x-worker-cmsd: - &worker-cmsd - image: "${QSERV_IMAGE:?err}" - init: true - # ports are published in worker-xrootd because this container uses that container's network stack. -x-worker-xrootd: - &worker-xrootd +x-worker-svc: + &worker-svc image: "${QSERV_IMAGE:?err}" init: true expose: - - "1094" - - "2131" - "3306" # for the worker db, which shares this container's network stack. x-repl-worker: &repl-worker @@ -53,14 +46,12 @@ volumes: volume_worker_0_data: volume_worker_0_results: - volume_worker_0_xrootd: volume_worker_0_home: volume_worker_0_mariadb_lib: volume_worker_0_mariadb_run: volume_worker_1_data: volume_worker_1_results: - volume_worker_1_xrootd: volume_worker_1_home: volume_worker_1_mariadb_lib: volume_worker_1_mariadb_run: @@ -97,29 +88,25 @@ services: - type: volume source: volume_worker_0_mariadb_run target: /var/run/mysqld # This is where the mariadb container puts the socket file - network_mode: "service:worker-xrootd-0" - worker-xrootd-0: - << : *worker-xrootd + network_mode: "service:worker-svc-0" + + worker-svc-0: + << : *worker-svc command: > - entrypoint worker-xrootd + entrypoint worker-svc --db-uri mysql://qsmaster:CHANGEME@127.0.0.1:3306 --db-admin-uri mysql://root:CHANGEME@127.0.0.1:3306 - --vnid-config "@/usr/local/lib64/libreplica.so {{db_uri}}/qservw_worker 0 0" --repl-instance-id qserv_proj --repl-auth-key replauthkey --repl-admin-auth-key=repladminauthkey --repl-registry-host repl-registry --repl-registry-port 25082 --results-dirname /qserv/data/results - --cmsd-manager-name czar-xrootd - --log-cfg-file=/config-etc/log/log-worker-xrootd.cnf + --log-cfg-file=/config-etc/log/log-worker-svc.cnf volumes: - type: volume source: volume_worker_0_results target: /qserv/data/results - - type: volume - source: volume_worker_0_xrootd - target: /var/run/xrootd - type: volume source: volume_worker_0_home target: /home/qserv @@ -130,36 +117,7 @@ services: networks: default: aliases: - - worker-cmsd-0 - worker-mariadb-0 - worker-cmsd-0: - << : *worker-cmsd - command: > - entrypoint worker-cmsd - --db-uri mysql://qsmaster:CHANGEME@worker-mariadb-0:3306 - --vnid-config "@/usr/local/lib64/libreplica.so mysql://qsmaster:CHANGEME@127.0.0.1:3306/qservw_worker 0 0" - --results-dirname /qserv/data/results - --repl-instance-id qserv_proj - --repl-auth-key replauthkey - --repl-admin-auth-key=repladminauthkey - --repl-registry-host repl-registry - --repl-registry-port 25082 - --cmsd-manager-name czar-xrootd - network_mode: "service:worker-xrootd-0" - volumes: - - type: volume - source: volume_worker_0_results - target: /qserv/data/results - - type: volume - source: volume_worker_0_xrootd - target: /var/run/xrootd - - type: volume - source: volume_worker_0_home - target: /home/qserv - - type: volume - source: volume_worker_0_mariadb_run - target: /qserv/mariadb/run # This matches the ?socket=... location in --db-uri and --db-admin-uri - - << : *log-volume repl-worker-0: << : *repl-worker command: > @@ -184,6 +142,7 @@ services: source: volume_worker_0_home target: /home/qserv - << : *log-volume + # worker 1 uses and validates socket file (where possible) to connect to the worker-mariadb worker-mariadb-1: << : *worker-mariadb @@ -201,30 +160,26 @@ services: - type: volume source: volume_worker_1_mariadb_run target: /var/run/mysqld # This is where the mariadb container puts the socket file - network_mode: "service:worker-xrootd-1" - worker-xrootd-1: - << : *worker-xrootd + network_mode: "service:worker-svc-1" + + worker-svc-1: + << : *worker-svc command: > - entrypoint --log-level DEBUG worker-xrootd + entrypoint --log-level DEBUG worker-svc --db-uri mysql://qsmaster:CHANGEME@127.0.0.1:3306?socket={{db_socket}} --db-admin-uri mysql://root:CHANGEME@127.0.0.1:3306?socket={{db_socket}} - --vnid-config "@/usr/local/lib64/libreplica.so mysql://qsmaster:CHANGEME@127.0.0.1:3306/qservw_worker 0 0" --repl-instance-id qserv_proj --repl-auth-key replauthkey --repl-admin-auth-key=repladminauthkey --repl-registry-host repl-registry --repl-registry-port 25082 --results-dirname /qserv/data/results - --cmsd-manager-name czar-xrootd --targs db_socket=/qserv/mariadb/run/mysqld.sock - --log-cfg-file=/config-etc/log/log-worker-xrootd.cnf + --log-cfg-file=/config-etc/log/log-worker-svc.cnf volumes: - type: volume source: volume_worker_1_results target: /qserv/data/results - - type: volume - source: volume_worker_1_xrootd - target: /var/run/xrootd - type: volume source: volume_worker_1_home target: /home/qserv @@ -235,36 +190,7 @@ services: networks: default: aliases: - - worker-cmsd-1 - worker-mariadb-1 - worker-cmsd-1: - << : *worker-cmsd - command: > - entrypoint --log-level DEBUG worker-cmsd - --db-uri mysql://qsmaster:CHANGEME@worker-mariadb-1:3306?socket=/qserv/mariadb/run/mysqld.sock - --vnid-config "@/usr/local/lib64/libreplica.so mysql://qsmaster:CHANGEME@127.0.0.1:3306/qservw_worker 0 0" - --results-dirname /qserv/data/results - --repl-instance-id qserv_proj - --repl-auth-key replauthkey - --repl-admin-auth-key=repladminauthkey - --repl-registry-host repl-registry - --repl-registry-port 25082 - --cmsd-manager-name czar-xrootd - network_mode: "service:worker-xrootd-1" - volumes: - - type: volume - source: volume_worker_1_results - target: /qserv/data/results - - type: volume - source: volume_worker_1_xrootd - target: /var/run/xrootd - - type: volume - source: volume_worker_1_home - target: /home/qserv - - type: volume - source: volume_worker_1_mariadb_run - target: /qserv/mariadb/run - - << : *log-volume repl-worker-1: << : *repl-worker # qserv-replica-worker app does not support socket file yet. @@ -290,42 +216,7 @@ services: source: volume_worker_1_home target: /home/qserv - << : *log-volume - czar-xrootd: - image: "${QSERV_IMAGE:?err}" - init: true - command: > - entrypoint xrootd-manager - --cmsd-manager-name czar-xrootd - hostname: czar-xrootd - expose: - - "1094" - - "2131" - volumes: - - type: volume - source: volume_czar_xrootd - target: /var/run/xrootd - - type: volume - source: volume_worker_1_home - target: /home/qserv - - << : *log-volume - networks: - default: - aliases: - - czar-cmsd - czar-cmsd: - image: "${QSERV_IMAGE:?err}" - init: true - # NOTE!! cms-delay-servers must match the number of workers being launched! - command: entrypoint cmsd-manager --cms-delay-servers 2 - network_mode: "service:czar-xrootd" - volumes: - - type: volume - source: volume_czar_xrootd - target: /var/run/xrootd - - type: volume - source: volume_czar_home - target: /home/qserv - - << : *log-volume + czar-mariadb: image: "${QSERV_MARIADB_IMAGE:?err}" init: true @@ -349,6 +240,7 @@ services: - type: volume source: volume_czar_mariadb_run target: /var/run/mysqld + czar-proxy: image: "${QSERV_IMAGE:?err}" init: true @@ -357,7 +249,6 @@ services: --db-uri mysql://qsmaster:CHANGEME@127.0.0.1:3306?socket={{db_socket}} --db-admin-uri mysql://root:CHANGEME@127.0.0.1:3306?socket={{db_socket}} --targs db_socket=/qserv/mariadb/run/mysqld.sock - --xrootd-manager czar-xrootd --log-cfg-file=/config-etc/log/log-czar-proxy.cnf --repl-instance-id qserv_proj --repl-auth-key replauthkey @@ -393,7 +284,6 @@ services: command: > entrypoint --log-level DEBUG czar-http --db-uri mysql://qsmaster:CHANGEME@czar-mariadb:3306/ - --xrootd-manager czar-xrootd --czar-name http --http-port 4048 --http-threads 4 @@ -453,7 +343,6 @@ services: --instance-id=qserv_proj --auth-key=replauthkey --admin-auth-key=repladminauthkey - --xrootd-host=czar-xrootd --registry-host=repl-registry --controller-auto-register-workers=1 --qserv-sync-force diff --git a/deploy/compose/log/log-czar-proxy.cnf b/deploy/compose/log/log-czar-proxy.cnf index c8c8b11b54..be40eb2660 100644 --- a/deploy/compose/log/log-czar-proxy.cnf +++ b/deploy/compose/log/log-czar-proxy.cnf @@ -4,5 +4,3 @@ log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-ddTHH:mm:ss.SSSZ} LWP %-5X{LWP} %-5p %m%n -log4j.logger.lsst.qserv.xrdssi.msgs=WARN -#log4j.logger.lsst.qserv.xrdssi.msgs=DEBUG diff --git a/deploy/compose/log/log-worker-xrootd.cnf b/deploy/compose/log/log-worker-svc.cnf similarity index 87% rename from deploy/compose/log/log-worker-xrootd.cnf rename to deploy/compose/log/log-worker-svc.cnf index 5d6ec716e2..be40eb2660 100644 --- a/deploy/compose/log/log-worker-xrootd.cnf +++ b/deploy/compose/log/log-worker-svc.cnf @@ -4,4 +4,3 @@ log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-ddTHH:mm:ss.SSSZ} LWP %-5X{LWP} %-5p %m%n -log4j.logger.lsst.qserv.xrdssi.msgs=WARN diff --git a/deploy/compose/log/log.cnf b/deploy/compose/log/log.cnf index 3ef039121b..be40eb2660 100644 --- a/deploy/compose/log/log.cnf +++ b/deploy/compose/log/log.cnf @@ -4,6 +4,3 @@ log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-ddTHH:mm:ss.SSSZ} LWP %-5X{LWP} %-5p %m%n -log4j.logger.lsst.qserv.xrdssi.msgs=WARN -#log4j.logger.lsst.qserv.xrdssi.msgs=DEBUG - diff --git a/deploy/docker/base/Dockerfile b/deploy/docker/base/Dockerfile index 4e4f67e729..ecd145a256 100644 --- a/deploy/docker/base/Dockerfile +++ b/deploy/docker/base/Dockerfile @@ -1,7 +1,7 @@ #------------------------------------------------------------------------------------------------------------- # The 'lite-build' target builds a container image to be used as the base of the supported build environment # for Qserv binaries. It includes the compiler toolchain and other developer tools, system libraries, and -# relatively-slowly changing third-party libs and tools (boost, antlr, protobuf, mysql-proxy, xrootd, etc.) +# relatively-slowly changing third-party libs and tools (boost, antlr, protobuf, mysql-proxy, etc.) # # The idea is that this container should change relatively infrequently (i.e. every few months or so), and # that developers will be able to pull and cache the latest version generated by CI. Developers should not @@ -133,20 +133,6 @@ RUN cd /tmp \ && cd /tmp \ && rm -rf mysqlproxy -RUN cd /tmp \ - && git clone https://github.com/xrootd/xrootd.git \ - && cd xrootd \ - && git checkout tags/v5.9.0 \ - && git config --global user.email "qserv@slac.stanford.edu" \ - && git config --global user.name "Qserv" \ - && mkdir build \ - && cd build \ - && cmake -DENABLE_PYTHON=off .. \ - && make -j8 \ - && make install \ - && cd /tmp \ - && rm -rf xrootd - RUN cd /tmp \ && git clone https://github.com/yhirose/cpp-httplib.git \ && cd cpp-httplib \ @@ -269,8 +255,7 @@ RUN mkdir -p /qserv/data && \ mkdir /config-etc && \ mkdir /config-etc/ssl && \ mkdir -p /qserv/run/tmp && \ - mkdir -p /var/run/xrootd && \ - chown qserv:qserv /qserv/data /config-etc /config-etc/ssl /qserv/run/tmp /var/run/xrootd + chown qserv:qserv /qserv/data /config-etc /config-etc/ssl /qserv/run/tmp RUN alternatives --install /usr/bin/python3 python3 /usr/bin/python3.12 1 ENV PYTHONPATH "${PYTHONPATH}:/usr/local/python" @@ -286,11 +271,6 @@ COPY --from=lite-build /usr/local/lib/libmysql-*.so /usr/local/lib/ COPY --from=lite-build /usr/local/lib/mysql-proxy/lua/*.so /usr/local/lib/mysql-proxy/lua/ COPY --from=lite-build /usr/local/lib/mysql-proxy/plugins/*.so /usr/local/lib/mysql-proxy/plugins/ -COPY --from=lite-build /usr/local/include/xrootd/ /usr/local/include/ -COPY --from=lite-build /usr/local/lib64/libXrd*.so /usr/local/lib64/ -COPY --from=lite-build /usr/local/bin/xrootd /usr/local/bin/ -COPY --from=lite-build /usr/local/bin/cmsd /usr/local/bin/ -COPY --from=lite-build /usr/local/bin/xrdfs /usr/local/bin/ COPY --from=lite-build /usr/local/lib64/libcpp-httplib.so /usr/local/lib64/ COPY --from=lite-build /usr/local/lib64/libaws-*.so /usr/local/lib64/ diff --git a/etc/log.cnf b/etc/log.cnf index 457d881474..ec675b61a3 100644 --- a/etc/log.cnf +++ b/etc/log.cnf @@ -3,4 +3,3 @@ log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-ddTHH:mm:ss.SSSZ} LWP %-5X{LWP} %-5p %m%n -log4j.logger.lsst.qserv.xrdssi.msgs=WARN diff --git a/python/lsst/qserv/admin/cli/entrypoint.py b/python/lsst/qserv/admin/cli/entrypoint.py index 9145a3f8eb..a80b183d59 100644 --- a/python/lsst/qserv/admin/cli/entrypoint.py +++ b/python/lsst/qserv/admin/cli/entrypoint.py @@ -40,7 +40,6 @@ OptionGroup, option_case, option_cmd, - option_cmsd_manager_name, option_compare_results, option_czar_connection, option_db_admin_uri, @@ -68,9 +67,7 @@ option_targs, option_tests_yaml, option_unload, - option_vnid_config, option_worker_connection, - option_xrootd_manager, options_targs, ) from .render_targs import render_targs @@ -82,18 +79,12 @@ mysql_proxy_cfg_template = os.path.join(template_dir, "proxy/etc/my-proxy.cnf.jinja") czar_cfg_template = os.path.join(template_dir, "proxy/etc/qserv-czar.cnf.jinja") czar_http_cfg_template = os.path.join(template_dir, "http/etc/qserv-czar.cnf.jinja") -cmsd_manager_cfg_template = os.path.join(template_dir, "xrootd/etc/cmsd-manager.cf.jinja") -cmsd_worker_cfg_template = os.path.join(template_dir, "xrootd/etc/cmsd-worker.cf.jinja") -xrdssi_cfg_template = os.path.join(template_dir, "xrootd/etc/xrdssi.cf.jinja") -xrootd_manager_cfg_template = os.path.join(template_dir, "xrootd/etc/xrootd-manager.cf.jinja") +worker_svc_cfg_template = os.path.join(template_dir, "worker-svc/etc/worker-svc.cf.jinja") mysql_proxy_cfg_path = "/config-etc/my-proxy.cnf" czar_cfg_path = "/config-etc/qserv-czar.cnf" czar_http_cfg_path = "/config-etc/qserv-czar.cnf" -cmsd_manager_cfg_path = "/config-etc/cmsd-manager.cnf" -cmsd_worker_cfg_path = "/config-etc/cmsd-worker.cf" -xrdssi_cfg_path = "/config-etc/xrdssi-worker.cf" -xrootd_manager_cfg_path = "/config-etc/xrootd-manager.cf" +worker_svc_cfg_path = "/config-etc/worker-svc.cf" socket_option_help = f"""Accepts query key { click.style("socket", bold=True) @@ -117,7 +108,7 @@ worker_db_help = f"""Non-admin URI to the worker database. {socket_option_help} - Populates 'hostname', 'port', and 'socket' under '[mysql]' in the xrdssi config + Populates 'hostname', 'port', and 'socket' under '[mysql]' in the worker config file. Also used to wait for schema to be at the correct version in this database. """ @@ -160,20 +151,6 @@ class CommandInfo: "--verbose", ), ), - ( - "cmsd-manager", - CommandInfo( - "cmsd -c {{cmsd_manager_cfg_path}} -n manager -I v4", - ), - ), - ("xrootd-manager", CommandInfo("xrootd -c {{xrootd_manager_cfg_path}} -n manager -I v4")), - ( - "worker-cmsd", - CommandInfo( - "cmsd -c {{cmsd_worker_cfg_path}} -n worker -I v4 -l @libXrdSsiLog.so -+xrdssi " - "{{xrdssi_cfg_path}}", - ), - ), ( "worker-repl", CommandInfo( @@ -183,10 +160,9 @@ class CommandInfo: ), ), ( - "worker-xrootd", + "worker-svc", CommandInfo( - "xrootd -c {{cmsd_worker_cfg_path}} -n worker -I v4 -l @libXrdSsiLog.so -+xrdssi " - "{{xrdssi_cfg_path}}", + "qserv-worker-http -v -c {{worker_svc_cfg_path}} -n worker", ), ), ( @@ -219,38 +195,20 @@ class CommandInfo: ) -option_cmsd_worker_cfg_file = partial( +option_worker_svc_cfg_file = partial( click.option, - "--cmsd-worker-cfg-file", - help="Path to the cmsd worker config file.", - default=cmsd_worker_cfg_template, + "--worker-svc-cfg-file", + help="Path to the worker-svc config file.", + default=worker_svc_cfg_template, show_default=True, ) -option_cmsd_worker_cfg_path = partial( +option_worker_svc_cfg_path = partial( click.option, - "--cmsd-worker-cfg-path", - help="Location to render cmsd_worker_cfg_file.", - default=cmsd_worker_cfg_path, - show_default=True, -) - - -option_xrdssi_cfg_file = partial( - click.option, - "--xrdssi-cfg-file", - help="Path to the xrdssi config file.", - default=xrdssi_cfg_template, - show_default=True, -) - - -option_xrdssi_cfg_path = partial( - click.option, - "--xrdssi-cfg-path", - help="Location to render xrdssi-cfg-file.", - default=xrdssi_cfg_path, + "--worker-svc-cfg-path", + help="Location to render worker-svc-cfg-file.", + default=worker_svc_cfg_path, show_default=True, ) @@ -560,7 +518,6 @@ def delete_database( help="The admin URI to the proxy's database, used for schema initialization. " + socket_option_help, required=True, ) -@option_xrootd_manager(required=True) @click.option( "--proxy-backend-address", default="127.0.0.1:3306", @@ -626,7 +583,6 @@ def proxy(ctx: click.Context, **kwargs: Any) -> None: help="The non-admin URI to the Czar's database, used for non-smig purposes. " + socket_option_help, required=True, ) -@option_xrootd_manager(required=True) @click.option( "--http-port", default="4048", @@ -732,112 +688,11 @@ def czar_http(ctx: click.Context, **kwargs: Any) -> None: ) -@entrypoint.command() -@pass_context -@click.option( - "--cms-delay-servers", - help="Populates 'cms.delay servers' in the cmsd manager config file.", -) -@click.option( - "--cmsd_manager_cfg_file", - help="Path to the cmsd manager config file.", - default=cmsd_manager_cfg_template, - show_default=True, -) -@click.option( - "--cmsd-manager-cfg-path", - help="Location to render cmsd_manager_cfg_file", - default=cmsd_manager_cfg_path, - show_default=True, -) -@options_targs() -@options_cms() -@option_options_file() -def cmsd_manager(ctx: click.Context, **kwargs: Any) -> None: - """Start as a cmsd manager node.""" - targs = utils.targs(ctx) - targs = render_targs(targs) - script.enter_manager_cmsd( - targs=targs, - cmsd_manager_cfg_file=targs["cmsd_manager_cfg_file"], - cmsd_manager_cfg_path=targs["cmsd_manager_cfg_path"], - cmd=targs["cmd"], - ) - - -@entrypoint.command() -@pass_context -@option_cmsd_manager_name() -@click.option( - "--xrootd_manager-cfg-file", - help="Path to the xrootd manager config file.", - default=xrootd_manager_cfg_template, - show_default=True, -) -@click.option( - "--xrootd-manager-cfg-path", - help="Location to render xrootd_manager_cfg_file.", - default=xrootd_manager_cfg_path, - show_default=True, -) -@options_targs() -@options_cms() -@option_options_file() -def xrootd_manager(ctx: click.Context, **kwargs: Any) -> None: - """Start as an xrootd manager node.""" - targs = utils.targs(ctx) - targs = render_targs(targs) - script.enter_xrootd_manager( - targs=targs, - xrootd_manager_cfg_file=targs["xrootd_manager_cfg_file"], - xrootd_manager_cfg_path=targs["xrootd_manager_cfg_path"], - cmd=targs["cmd"], - ) - - -@entrypoint.command(help=f"Start as a worker cmsd node.\n\n{socket_option_description}") -@pass_context -@option_db_uri(help=worker_db_help) -@option_vnid_config(required=True) -@option_vnid_config(required=True) -@option_repl_instance_id(required=True) -@option_repl_auth_key(required=True) -@option_repl_admin_auth_key(required=True) -@option_repl_registry_host(required=True) -@option_repl_registry_port(required=True) -@option_repl_http_port(required=True) -@option_results_dirname() -@option_cmsd_manager_name() -@option_debug() -@option_cmsd_worker_cfg_file() -@option_cmsd_worker_cfg_path() -@option_xrdssi_cfg_file() -@option_xrdssi_cfg_path() -@option_log_cfg_file() -@options_targs() -@options_cms() -@option_options_file() -def worker_cmsd(ctx: click.Context, **kwargs: Any) -> None: - targs = utils.targs(ctx) - targs = render_targs(targs) - script.enter_worker_cmsd( - targs=targs, - db_uri=targs["db_uri"], - cmsd_worker_cfg_file=targs["cmsd_worker_cfg_file"], - cmsd_worker_cfg_path=targs["cmsd_worker_cfg_path"], - xrdssi_cfg_file=targs["xrdssi_cfg_file"], - xrdssi_cfg_path=targs["xrdssi_cfg_path"], - log_cfg_file=targs["log_cfg_file"], - cmd=targs["cmd"], - ) - - -@entrypoint.command(help=f"Start as a worker xrootd node.\n\n{socket_option_description}") +@entrypoint.command(help=f"Start as a worker-svc node.\n\n{socket_option_description}") @pass_context @option_debug() @option_db_uri(help=worker_db_help) @option_db_admin_uri(help=admin_worker_db_help) -@option_vnid_config(required=True) @option_repl_instance_id(required=True) @option_repl_auth_key(required=True) @option_repl_admin_auth_key(required=True) @@ -845,27 +700,22 @@ def worker_cmsd(ctx: click.Context, **kwargs: Any) -> None: @option_repl_registry_port(required=True) @option_repl_http_port(required=True) @option_results_dirname() -@option_cmsd_manager_name() @option_db_qserv_user() -@option_cmsd_worker_cfg_file() -@option_cmsd_worker_cfg_path() -@option_xrdssi_cfg_file() -@option_xrdssi_cfg_path() +@option_worker_svc_cfg_file() +@option_worker_svc_cfg_path() @option_log_cfg_file() @options_targs() @options_cms() @option_options_file() -def worker_xrootd(ctx: click.Context, **kwargs: Any) -> None: +def worker_svc(ctx: click.Context, **kwargs: Any) -> None: targs = utils.targs(ctx) targs = render_targs(targs) - script.enter_worker_xrootd( + script.enter_worker_svc( targs=targs, db_uri=targs["db_uri"], db_admin_uri=targs["db_admin_uri"], - cmsd_worker_cfg_file=targs["cmsd_worker_cfg_file"], - cmsd_worker_cfg_path=targs["cmsd_worker_cfg_path"], - xrdssi_cfg_file=targs["xrdssi_cfg_file"], - xrdssi_cfg_path=targs["xrdssi_cfg_path"], + worker_svc_cfg_file=targs["worker_svc_cfg_file"], + worker_svc_cfg_path=targs["worker_svc_cfg_path"], log_cfg_file=targs["log_cfg_file"], cmd=targs["cmd"], ) @@ -921,10 +771,6 @@ def worker_repl(ctx: click.Context, **kwargs: Any) -> None: + socket_option_help, required=True, ) -@click.option( - "--xrootd-manager", - help="The host name of the xrootd manager node.", -) @option_log_cfg_file() @options_cms() @click.option( diff --git a/python/lsst/qserv/admin/cli/options.py b/python/lsst/qserv/admin/cli/options.py index 1b7d95bbcd..987cbffd5f 100644 --- a/python/lsst/qserv/admin/cli/options.py +++ b/python/lsst/qserv/admin/cli/options.py @@ -192,21 +192,6 @@ def __call__(self, f: Callable) -> Callable: ) -option_vnid_config = partial( - click.option, - "--vnid-config", - help="The config parameters used by the qserv cmsd to get the vnid from the specified " - " source (static string, a file or worker database).", -) - - -option_xrootd_manager = partial( - click.option, - "--xrootd-manager", - help="The host name of the xrootd manager.", -) - - option_tests_yaml = partial( click.option, "--tests-yaml", diff --git a/python/lsst/qserv/admin/cli/script.py b/python/lsst/qserv/admin/cli/script.py index 0d512feb3a..c7d2950559 100644 --- a/python/lsst/qserv/admin/cli/script.py +++ b/python/lsst/qserv/admin/cli/script.py @@ -266,136 +266,16 @@ def smig_worker(connection: str, update: bool) -> None: _do_smig(worker_smig_dir, "worker", connection, update) -def enter_manager_cmsd( - targs: Targs, - cmsd_manager_cfg_file: str, - cmsd_manager_cfg_path: str, - cmd: str, -) -> None: - """Start a cmsd manager qserv node. - - Parameters - ---------- - targs : `Targs` - The arguments for template expansion. - cmsd_manager_cfg_file : str - Path to the cmsd manager config file. - cmsd_manager_cfg_path : str - Location to render cmsd_manager_cfg_template. - cmd : str - The jinja2 template for the command for this function to execute. - """ - apply_template_cfg_file(cmsd_manager_cfg_file, cmsd_manager_cfg_path, targs) - - env = dict( - os.environ, - LD_PRELOAD=ld_preload, - ) - - sys.exit(_run(args=None, env=env, cmd=cmd)) - - -def enter_xrootd_manager( - targs: Targs, - xrootd_manager_cfg_file: str, - xrootd_manager_cfg_path: str, - cmd: str, -) -> None: - """Start an xrootd manager qserv node. - - Parameters - ---------- - targs : Targs - The arguments for template expansion. - xrootd_manager_cfg_file : str - Path to the cmsd manager config file. - xrootd_manager_cfg_path : str - Location to render cmsd_manager_cfg_template. - cmd : str - The jinja2 template for the command for this function to execute. - """ - apply_template_cfg_file(xrootd_manager_cfg_file, xrootd_manager_cfg_path, targs) - - env = dict( - os.environ, - LD_PRELOAD=ld_preload, - ) - - sys.exit(_run(args=None, env=env, cmd=cmd)) - - -def enter_worker_cmsd( - targs: Targs, - db_uri: str, - cmsd_worker_cfg_file: str, - cmsd_worker_cfg_path: str, - xrdssi_cfg_file: str, - xrdssi_cfg_path: str, - log_cfg_file: str, - cmd: str, -) -> None: - """Start a worker cmsd node. - - Parameters - ---------- - vnid_config : str - The config parameters used by the qserv cmsd to get the vnid - from the specified source (static string, a file or worker database). - targs : Targs - The arguments for template expansion. - db_uri : str - The non-admin URI to the worker's database. - cmsd_worker_cfg_file : str - The path to the worker cmsd config file. - cmsd_worker_cfg_path : str - The location to render the worker cmsd config file. - xrdssi_cfg_file : str - The path to the xrdssi config file. - xrdssi_cfg_path : str - The location to render the the xrdssi config file. - log_cfg_file : `str` - Location of the log4cxx config file. - cmd : str - The jinja2 template for the command for this function to execute. - """ - url = _process_uri( - uri=db_uri, - query_keys=("socket",), - option=options.option_db_uri.args[0], - block=True, - ) - targs["db_host"] = url.host - targs["db_port"] = url.port or "" - targs["db_socket"] = url.query.get("socket", "") - - apply_template_cfg_file(cmsd_worker_cfg_file, cmsd_worker_cfg_path, targs) - apply_template_cfg_file(xrdssi_cfg_file, xrdssi_cfg_path, targs) - - # wait before worker database will be fully initialized as needed - # for the vnid plugin to function correctly - _do_smig_block(worker_smig_dir, "worker", db_uri) - - env = dict( - os.environ, - LD_PRELOAD=ld_preload, - LSST_LOG_CONFIG=log_cfg_file, - ) - - sys.exit(_run(args=None, env=env, cmd=cmd)) - - -def enter_worker_xrootd( +def enter_worker_svc( targs: Targs, db_uri: str, db_admin_uri: str, - cmsd_worker_cfg_file: str, - cmsd_worker_cfg_path: str, - xrdssi_cfg_file: str, - xrdssi_cfg_path: str, + worker_svc_cfg_file: str, + worker_svc_cfg_path: str, log_cfg_file: str, cmd: str, ) -> None: - """Start a worker xrootd node. + """Start a worker wkr node. Parameters ---------- @@ -405,14 +285,10 @@ def enter_worker_xrootd( The non-admin URI to the proxy's database. db_admin_uri : str The admin URI to the proxy's database. - cmsd_worker_cfg_file : str - The path to the worker cmsd config file. - cmsd_worker_cfg_path : str - The location to render to the worker cmsd config file. - xrdssi_cfg_file : str - The path to the xrdssi config file. - xrdssi_cfg_path : str - The location to render to the xrdssi config file. + worker_svc_cfg_file : str + The path to the worker config file. + worker_svc_cfg_path : str + The location to render to the worker config file. log_cfg_file : `str` Location of the log4cxx config file. cmd : `str` @@ -453,11 +329,7 @@ def enter_worker_xrootd( smig_worker(db_admin_uri, update=False) - # TODO worker (and manager) xrootd+cmsd pair should "share" the cfg file - # it's in different containers but should be same source & processing. - # Rename these files to be more agnostic. - apply_template_cfg_file(cmsd_worker_cfg_file, cmsd_worker_cfg_path) - apply_template_cfg_file(xrdssi_cfg_file, xrdssi_cfg_path) + apply_template_cfg_file(worker_svc_cfg_file, worker_svc_cfg_path) env = dict( os.environ, diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 70540f15a2..ee58899b2e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -21,7 +21,6 @@ find_package(Protobuf REQUIRED) find_package(Python REQUIRED COMPONENTS Development Interpreter) find_package(pybind11 REQUIRED) find_package(Threads REQUIRED) -find_package(XRootD REQUIRED) find_package(httplib REQUIRED) find_package(aws-c-auth REQUIRED) find_package(aws-c-cal REQUIRED) @@ -79,13 +78,12 @@ add_subdirectory(rproc) add_subdirectory(sql) add_subdirectory(util) add_subdirectory(wbase) +add_subdirectory(wcomms) add_subdirectory(wconfig) add_subdirectory(wcontrol) add_subdirectory(wdb) +add_subdirectory(wmain) add_subdirectory(wpublish) add_subdirectory(wsched) add_subdirectory(www) -add_subdirectory(xrdlog) -add_subdirectory(xrdsvc) - diff --git a/src/admin/templates/http/etc/qserv-czar.cnf.jinja b/src/admin/templates/http/etc/qserv-czar.cnf.jinja index e996142eef..b2d14f7cd1 100644 --- a/src/admin/templates/http/etc/qserv-czar.cnf.jinja +++ b/src/admin/templates/http/etc/qserv-czar.cnf.jinja @@ -1,6 +1,5 @@ [frontend] -xrootd={{ xrootd_manager }}:1094 #[mgmtdb] #db=qservMeta @@ -55,26 +54,6 @@ port = {{ czar_db_port }} [tuning] largeResultConcurrentMerges = 6 -# xrootdCBThreadsInit must be less than xrootdCBThreadsMax -xrootdCBThreadsMax = 500 -xrootdCBThreadsInit = 50 - -# INSTRUCTIONS: -# -# Allowed range: 1 - 1024. Any number above 1024 will be truncated to 1024. -# The deafault value of the parameter in xrootd is 4. Set it higher in -# Qserv deployments with the large number of chunks per catalog, or/and -# for use cases where many queries are processed by Qserv simultaneously, -# even for a modest number of chunks per catalog. -# -# Set the parameter to 0 to enable auto-tuning. -# -# The general formula for auto-tuning implies incrementing a value of the parameter -# by 1 for every 65,000 "in-flight" requests. -# -# The value set below has been tested to work for 80 simultaneous "shared scan" queries -# of 150,000 chunks each. -xrootdSpread = 0 # Seconds between updates the czar sends to qmeta for completed chunks. # This is per user query and important milestones ignore this limit. diff --git a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja index 49dc123c61..1856bcc6a3 100644 --- a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja +++ b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja @@ -1,6 +1,5 @@ [frontend] -xrootd={{ xrootd_manager }}:1094 #[mgmtdb] #db=qservMeta @@ -57,26 +56,6 @@ port = {{ czar_db_port }} [tuning] #largeResultConcurrentMerges = 3 largeResultConcurrentMerges = 6 -# xrootdCBThreadsInit must be less than xrootdCBThreadsMax -xrootdCBThreadsMax = 500 -xrootdCBThreadsInit = 50 - -# INSTRUCTIONS: -# -# Allowed range: 1 - 1024. Any number above 1024 will be truncated to 1024. -# The deafault value of the parameter in xrootd is 4. Set it higher in -# Qserv deployments with the large number of chunks per catalog, or/and -# for use cases where many queries are processed by Qserv simultaneously, -# even for a modest number of chunks per catalog. -# -# Set the parameter to 0 to enable auto-tuning. -# -# The general formula for auto-tuning implies incrementing a value of the parameter -# by 1 for every 65,000 "in-flight" requests. -# -# The value set below has been tested to work for 80 simultaneous "shared scan" queries -# of 150,000 chunks each. -xrootdSpread = 0 # Seconds between updates the czar sends to qmeta for completed chunks. # This is per user query and important milestones ignore this limit. diff --git a/src/admin/templates/xrootd/etc/xrdssi.cf.jinja b/src/admin/templates/worker-svc/etc/worker-svc.cf.jinja similarity index 95% rename from src/admin/templates/xrootd/etc/xrdssi.cf.jinja rename to src/admin/templates/worker-svc/etc/worker-svc.cf.jinja index 49c9987292..233b587d2a 100644 --- a/src/admin/templates/xrootd/etc/xrdssi.cf.jinja +++ b/src/admin/templates/worker-svc/etc/worker-svc.cf.jinja @@ -1,4 +1,4 @@ -# Qserv xrdssi plugin configuration file +# Qserv worker configuration file # Default values for parameters are commented [mysql] @@ -71,10 +71,6 @@ reservedinteractivesqlconn = 930 # The name of a folder where query results will be stored. dirname = {{ results_dirname }} -# The port number of the worker XROOTD service for serving files. -# NOTE: the hardcoded value may need to be replaced with a template -xrootd_port = 1094 - # The number of the BOOST ASIO threads for HTTP requests num_http_threads = 4 diff --git a/src/admin/templates/xrootd/etc/cmsd-manager.cf.jinja b/src/admin/templates/xrootd/etc/cmsd-manager.cf.jinja deleted file mode 100644 index f907bcd1e6..0000000000 --- a/src/admin/templates/xrootd/etc/cmsd-manager.cf.jinja +++ /dev/null @@ -1,49 +0,0 @@ -all.role manager - -# Path to write logging and other information -all.adminpath /var/run/xrootd - -# Do not change. This specifies valid virtual paths that can be accessed. -# "nolock" directive prevents write-locking and is important for qserv -# qserv is hardcoded for these paths. -all.export / nolock - -# Specify that no significant free space is required on servers -# Indeed current configuration doesn't expect to be dynamically -# written to, but export the space in R/W mode -cms.space 1k 2k - -# Specify the minimum number of servers that must be subscribed for load -# balancing to be effective. -cms.delay servers {{ cms_delay_servers }} - -# ssi.loglib libxrdlog.so - -# Optional: Prevent dns resolution in logs. -# This may speed up request processing. -xrd.network nodnr - -# This causes hostname resolution to occur at run-time not configuration time -# This is required by k8s -# Andy H. still have to modify the local IP-to-Name cache to account -# for dynamic DNS (it doesn't now). Unfortunately, it's a non-ABI compatible -# change so it will go into Release 5 branch not git master. The caching -# shouldn't really be a problem but if causes you grief simply turn it off by -# also specifying "xrd.network cache 0". Once Andy H. fixes the cache it will work -# correctly with a dynamic DNS with no side-effects (though it's unlikely any of -# them are observed as it is). -xrd.network dyndns -xrd.network cache 0 - -all.manager UNUSED:2131 - -# - cmsd redirector runs on port 2131 -# - cmsd server does not open server socket -# but only client connection to cmsd redirector -# - xrootd default port is 1094 -if exec cmsd - xrd.port 2131 -fi - -# Uncomment the following line for detailed xrootd debugging -# xrootd.trace all debug diff --git a/src/admin/templates/xrootd/etc/cmsd-worker.cf.jinja b/src/admin/templates/xrootd/etc/cmsd-worker.cf.jinja deleted file mode 100644 index 172c2edc51..0000000000 --- a/src/admin/templates/xrootd/etc/cmsd-worker.cf.jinja +++ /dev/null @@ -1,68 +0,0 @@ -# Use server mode -all.role server - -cms.vnid {{ vnid_config }} - -# Use XrdSsi plugin -xrootd.fslib -2 libXrdSsi.so -ssi.svclib libxrdsvc.so -oss.statlib -2 -arevents libXrdSsi.so - -# Force disable asyncronous access -# because of XrdSsi -xrootd.async off - -ssi.trace all debug - -######################################## -# Shared directives (manager and server) -######################################## - -# Path to write logging and other information -all.adminpath /var/run/xrootd - -# Do not change. This specifies valid virtual paths that can be accessed. -# "nolock" directive prevents write-locking and is important for qserv -# qserv is hardcoded for these paths. -all.export / nolock - -# Specify that no significant free space is required on servers -# Indeed current configuration doesn't expect to be dynamically -# written to, but export the space in R/W mode -cms.space 1k 2k - -ssi.loglib libxrdlog.so - -# Optional: Prevent dns resolution in logs. -# This may speed up request processing. -xrd.network nodnr - -# This causes hostname resolution to occur at run-time not configuration time -# This is required by k8s -# Andy H. still have to modify the local IP-to-Name cache to account -# for dynamic DNS (it doesn't now). Unfortunately, it's a non-ABI compatible -# change so it will go into Release 5 branch not git master. The caching -# shouldn't really be a problem but if causes you grief simply turn it off by -# also specifying "xrd.network cache 0". Once Andy H. fixes the cache it will work -# correctly with a dynamic DNS with no side-effects (though it's unlikely any of -# them are observed as it is). -xrd.network dyndns -xrd.network cache 0 - -all.manager {{cmsd_manager_name}}:2131 - -# - cmsd redirector runs on port 2131 -# - cmsd server does not open server socket -# but only client connection to cmsd redirector -# - xrootd default port is 1094 -xrd.port 1094 - -# Uncomment the following line for detailed xrootd debugging -# xrootd.trace all debug - -# Enforce the default limits for the number of threads created/managed by XROOTD. -# Formally, these defaults are supposed to be enforced by the implementation -# as stated in: https://xrootd.web.cern.ch/doc/dev57/xrd_config.htm#_Toc171719950 -# In reality, no limit is set. The problem was reported to the XROOT developers -# in: https://github.com/xrootd/xrootd/issues/2468 -xrd.sched mint 8 maxt 2048 avlt 512 idle 780 diff --git a/src/admin/templates/xrootd/etc/xrootd-manager.cf.jinja b/src/admin/templates/xrootd/etc/xrootd-manager.cf.jinja deleted file mode 100644 index 8972766c89..0000000000 --- a/src/admin/templates/xrootd/etc/xrootd-manager.cf.jinja +++ /dev/null @@ -1,43 +0,0 @@ -# Use manager mode -all.role manager - -# Path to write logging and other information -all.adminpath /var/run/xrootd - -# Do not change. This specifies valid virtual paths that can be accessed. -# "nolock" directive prevents write-locking and is important for qserv -# qserv is hardcoded for these paths. -all.export / nolock - -# Specify that no significant free space is required on servers -# Indeed current configuration doesn't expect to be dynamically -# written to, but export the space in R/W mode -cms.space 1k 2k - -# ssi.loglib libxrdlog.so - -# Optional: Prevent dns resolution in logs. -# This may speed up request processing. -xrd.network nodnr - -# This causes hostname resolution to occur at run-time not configuration time -# This is required by k8s -# Andy H. still have to modify the local IP-to-Name cache to account -# for dynamic DNS (it doesn't now). Unfortunately, it's a non-ABI compatible -# change so it will go into Release 5 branch not git master. The caching -# shouldn't really be a problem but if causes you grief simply turn it off by -# also specifying "xrd.network cache 0". Once Andy H. fixes the cache it will work -# correctly with a dynamic DNS with no side-effects (though it's unlikely any of -# them are observed as it is). -xrd.network dyndns -xrd.network cache 0 - -all.manager {{cmsd_manager_name}}:2131 - -# - cmsd redirector runs on port 2131 -# - cmsd server does not open server socket -# but only client connection to cmsd redirector -# - xrootd default port is 1094 - -# Uncomment the following line for detailed xrootd debugging -# xrootd.trace all debug diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 935206440c..e370929427 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -135,6 +135,7 @@ string readHttpFileAndMerge(lsst::qserv::qdisp::UberJob::Ptr const& uberJob, str // Restart the tracker to measure the reading performance of the next chunk of data. transmitRateTracker = make_unique>(reportFileRecvRate); }); + if (offset != fileSize) { throw runtime_error(context + "short read"); } @@ -241,20 +242,22 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber throw util::Bug(ERR_LOC, "already flushed"); } - if (fileSize == 0) { - return qdisp::MergeEndStatus(true); - } - - // After this final test the job's result processing can't be interrupted. - if (uberJob->isQueryCancelled()) { - return qdisp::MergeEndStatus(true); - } + if (fileSize == 0) return qdisp::MergeEndStatus(true); // Read from the http stream and push records into the CSV stream in a separate thread. // Note the fixed capacity of the stream which allows up to 2 records to be buffered // in the stream. This is enough to hide the latency of the HTTP connection and // the time needed to read the file. auto csvStream = mysql::CsvStream::create(2); + _csvStream = csvStream; + + // This must be after setting _csvStream to avoid cancelFileMerge() + // race issues, and it needs to be before the thread starts. + auto exec = uberJob->getExecutive(); + if (exec == nullptr || exec->getCancelled() || exec->isRowLimitComplete()) { + return qdisp::MergeEndStatus(true); + } + string fileReadErrorMsg; thread csvThread([uberJob, csvStream, fileUrl, fileSize, &fileReadErrorMsg]() { size_t bytesRead = 0; @@ -283,12 +286,17 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber }); // Attempt the actual merge. - bool const fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvStream); + bool fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvStream); if (!fileMergeSuccess) { LOGS(_log, LOG_LVL_WARN, __func__ << " merge failed"); util::Error const& err = _infileMerger->getError(); _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg(), util::ErrorCode::RESULT_IMPORT); } + if (csvStream->getContaminated()) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " merge stream contaminated"); + fileMergeSuccess = false; + _setError(ccontrol::MSG_RESULT_ERROR, "merge stream contaminated", util::ErrorCode::RESULT_IMPORT); + } csvThread.join(); if (!fileReadErrorMsg.empty()) { @@ -299,15 +307,21 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber qdisp::MergeEndStatus mergeEStatus(fileMergeSuccess && fileReadErrorMsg.empty()); if (!mergeEStatus.success) { - // This error check needs to come after the csvThread.join() to avoid race conditions. - if (csvStream->getBytesWritten() > 0) { - // There was a failure and bytes were written, result table is ruined. - mergeEStatus.contaminated = true; - } + // This error check needs to come after the csvThread.join() to ensure writing + // is finished. If any bytes were written, the result table is ruined. + mergeEStatus.contaminated = csvStream->getBytesWritten() > 0; } + return mergeEStatus; } +void MergingHandler::cancelFileMerge() { + auto csvStrm = _csvStream.lock(); + if (csvStrm != nullptr) { + csvStrm->cancel(); + } +} + void MergingHandler::_setError(int code, std::string const& msg, int errorState) { LOGS(_log, LOG_LVL_DEBUG, "_setError: code: " << code << ", message: " << msg); auto exec = _executive.lock(); diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index c27070831c..68bdb2f632 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -36,6 +36,10 @@ namespace lsst::qserv::http { class ClientConnPool; } // namespace lsst::qserv::http +namespace lsst::qserv::mysql { +class CsvStream; +} // namespace lsst::qserv::mysql + namespace lsst::qserv::qdisp { class Executive; class JobQuery; @@ -50,14 +54,9 @@ class InfileMerger; namespace lsst::qserv::ccontrol { /// MergingHandler is an implementation of a ResponseHandler that implements -/// czar-side knowledge of the worker's response protocol. It leverages XrdSsi's -/// API by pulling the exact number of bytes needed for the next logical -/// fragment instead of performing buffer size and offset -/// management. Fully-constructed protocol messages are then passed towards an -/// InfileMerger. -/// Do to the way the code works, MerginHandler is effectively single threaded. -/// The worker can only send the data for this job back over a single channel -/// and it can only send one transmit on that channel at a time. +/// czar-side knowledge of the worker's response protocol. +/// The czar collects a result file from the worker and merges that into +/// the query result table. class MergingHandler : public qdisp::ResponseHandler { public: typedef std::shared_ptr Ptr; @@ -70,7 +69,7 @@ class MergingHandler : public qdisp::ResponseHandler { /// @see ResponseHandler::flushHttp /// @see MerginHandler::_mergeHttp /// @see qdisp::MergeEndStatus - qdisp::MergeEndStatus flushHttp(std::string const& fileUrl, uint64_t fileSize) override; + qdisp::MergeEndStatus flushHttp(std::string const& fileUrl, std::uint64_t fileSize) override; /// @see ResponseHandler::flushHttpError void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; @@ -78,13 +77,16 @@ class MergingHandler : public qdisp::ResponseHandler { /// Signal an unrecoverable error condition. No further calls are expected. void errorFlush(std::string const& msg, int code) override; + /// Stop an ongoing file merge, if possible. + void cancelFileMerge() override; + /// Print a string representation of the receiver to an ostream std::ostream& print(std::ostream& os) const override; private: /// Call InfileMerger to do the work of merging this data to the result. qdisp::MergeEndStatus _mergeHttp(std::shared_ptr const& uberJob, - std::string const& fileUrl, uint64_t fileSize); + std::string const& fileUrl, std::uint64_t fileSize); /// Set error code and string. void _setError(int code, std::string const& msg, int errorState); @@ -102,6 +104,7 @@ class MergingHandler : public qdisp::ResponseHandler { std::string _wName{"~"}; ///< worker name std::weak_ptr _executive; ///< Weak pointer to the executive for errors. + std::weak_ptr _csvStream; ///< Weak pointer to cancel infile merge. }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 68aa2f324f..8238013221 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -337,12 +337,13 @@ void UserQuerySelect::buildAndSendUberJobs() { return; } - if (exec->getCancelled() || exec->getSuperfluous()) { - LOGS(_log, LOG_LVL_INFO, funcN << " executive cancelled."); - } - if (exec->getSuperfluous()) { LOGS(_log, LOG_LVL_INFO, funcN << " executive superfluous, result already found."); + return; + } + if (exec->getCancelled()) { + LOGS(_log, LOG_LVL_INFO, funcN << " executive cancelled."); + return; } // Only one thread should be generating UberJobs for this user query at any given time. @@ -491,10 +492,15 @@ void UserQuerySelect::buildAndSendUberJobs() { if (!missingChunks.empty()) { string errStr = funcN + " a worker could not be found for these chunks "; + int maxList = 0; for (auto const& chk : missingChunks) { errStr += to_string(chk) + ","; + if (++maxList > 50) { + errStr += " too many to show all."; + break; + } } - errStr += " they will be retried later."; + errStr += " All will be retried later. Total missing=" + to_string(missingChunks.size()); LOGS(_log, LOG_LVL_ERROR, errStr); } diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 276be00beb..67a5261a34 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -247,8 +247,8 @@ class Czar { /// problems in a way that would wedge the pool. This can /// probably be done fairly easily by having pools /// attached to ActiveWorker in _activeWorkerMap. - /// This was not possible in xrootd as the czar had - /// no reasonable way to know where Jobs were going. + /// Previously, the czar had no reasonable way to + /// know where Jobs were going. std::shared_ptr _qdispPool; /// Pool of http client connections for sending commands (UberJobs diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index c1ebad63eb..3f84b3b928 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -71,7 +71,7 @@ void CzarChunkMap::sortChunks(std::vector& chunksSortedBySize) { std::sort(chunksSortedBySize.begin(), chunksSortedBySize.end(), sortBySizeDesc); } -void CzarChunkMap::verify() { +void CzarChunkMap::verify(string const& familyName) const { auto&& wcMap = *_workerChunkMap; auto&& chunkMap = *_chunkMap; // Use a set to prevent duplicate ids caused by replication levels > 1. @@ -85,20 +85,22 @@ void CzarChunkMap::verify() { for (auto const& [chunkId, chunkDataPtr] : chunkMap) { if (chunkDataPtr == nullptr) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " chunkId=" << chunkId << " had nullptr"); + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " family=" << familyName << " chunkId=" << chunkId << " had nullptr"); ++errorCount; continue; } auto primeScanWkr = chunkDataPtr->_primaryScanWorker.lock(); if (primeScanWkr == nullptr) { LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " chunkId=" << chunkId << " missing primaryScanWorker"); + cName(__func__) << " family=" << familyName << " chunkId=" << chunkId + << " missing primaryScanWorker"); ++errorCount; continue; } if (primeScanWkr->_sharedScanChunkMap.find(chunkId) == primeScanWkr->_sharedScanChunkMap.end()) { LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " chunkId=" << chunkId + cName(__func__) << " family=" << familyName << " chunkId=" << chunkId << " should have been (and was not) in the sharedScanChunkMap for " << primeScanWkr->_workerId); ++errorCount; @@ -109,7 +111,8 @@ void CzarChunkMap::verify() { allChunkIds.erase(iter); } else { LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " chunkId=" << chunkId << " chunkId was not in allChunks list"); + cName(__func__) << " family=" << familyName << " chunkId=" << chunkId + << " chunkId was not in allChunks list"); ++errorCount; continue; } @@ -130,8 +133,10 @@ void CzarChunkMap::verify() { if (errorCount > 0) { // Original creation of the family map will keep re-reading until there are no problems. // _monitor will log this and keep using the old maps. - throw ChunkMapException(ERR_LOC, "verification failed with " + to_string(errorCount) + " errors"); + throw ChunkMapException(ERR_LOC, "verification failed with " + to_string(errorCount) + " errors " + + " family=" + familyName); } + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " family=" << familyName << " verified"); } string CzarChunkMap::dumpChunkMap(ChunkMap const& chunkMap) { @@ -261,7 +266,7 @@ bool CzarChunkMap::WorkerChunksData::isDead() { } _activeWorker = awMap->getActiveWorker(_workerId); if (_activeWorker == nullptr) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " activeWorker not found."); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " activeWorker not found."); return true; } } @@ -335,10 +340,11 @@ bool CzarFamilyMap::_read() { std::lock_guard gLock(_familyMapMtx); qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(_lastUpdateTime); if (_lastUpdateTime == qChunkMap.updateTime) { - LOGS(_log, LOG_LVL_DEBUG, - cName(__func__) << " no need to read " + // If "_lastUpdateTime == qChunkMap.updateTime", qChunkMap is empty. + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " no need to read last=" << util::TimeUtils::timePointToDateTimeString(_lastUpdateTime) - << " db=" << util::TimeUtils::timePointToDateTimeString(qChunkMap.updateTime)); + << " map=" << util::TimeUtils::timePointToDateTimeString(qChunkMap.updateTime)); return false; } @@ -353,6 +359,10 @@ bool CzarFamilyMap::_read() { _lastUpdateTime = qChunkMap.updateTime; + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " read and verified " + << util::TimeUtils::timePointToDateTimeString(_lastUpdateTime)); + LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_read() end"); return true; } @@ -489,7 +499,7 @@ void CzarFamilyMap::insertIntoMaps(std::shared_ptr const& newFami void CzarFamilyMap::verify(std::shared_ptr const& familyMap) { for (auto&& [familyName, czarChunkMapPtr] : *familyMap) { - czarChunkMapPtr->verify(); + czarChunkMapPtr->verify(familyName); } } diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index a167ba0a2c..f3d9365327 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -85,7 +85,7 @@ class CzarChunkMap { using Ptr = std::shared_ptr; using SizeT = uint64_t; - std::string cName(const char* func) { return std::string("CzarChunkMap::") + func; } + std::string cName(const char* func) const { return std::string("CzarChunkMap::") + func; } CzarChunkMap(CzarChunkMap const&) = delete; CzarChunkMap& operator=(CzarChunkMap const&) = delete; @@ -103,7 +103,7 @@ class CzarChunkMap { using Ptr = std::shared_ptr; ChunkData(int chunkId_) : _chunkId(chunkId_) {} - std::string cName(const char* func) { + std::string cName(const char* func) const { return std::string("ChunkData::") + func + " " + std::to_string(_chunkId); } int64_t getChunkId() const { return _chunkId; } @@ -203,7 +203,7 @@ class CzarChunkMap { /// Verify that all chunks belong to at least one worker and that all chunks are represented in shared /// scans. /// @throws ChunkMapException - void verify(); + void verify(std::string const& familyName) const; static std::string dumpChunkMap(ChunkMap const& chunkMap); diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index d0f23ef666..01585ef8c3 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -142,7 +142,8 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { } json HttpCzarWorkerModule::_handleJobReady(string const& func) { - LOGS(_log, LOG_LVL_DEBUG, "HttpCzarWorkerModule::_handleJobReady start"); + string const fName = "HttpCzarWorkerModule::_handleJobReady"; + LOGS(_log, LOG_LVL_DEBUG, fName << " start"); // Metadata-only responses for the file-based protocol should not have any data // Parse and verify the json message and then have the uberjob import the file. @@ -157,12 +158,16 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { auto uberJobId = jrMsg->getUberJobId(); qdisp::Executive::Ptr exec = czar::Czar::getCzar()->getExecutiveFromMap(queryId); if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, + fName << " null exec QID:" << queryId << " ujId=" << uberJobId << " cz=" << czarId); throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + to_string(queryId) + " czar=" + to_string(czarId)); } qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); if (uj == nullptr) { + LOGS(_log, LOG_LVL_WARN, + fName << " null uj QID:" << queryId << " ujId=" << uberJobId << " cz=" << czarId); throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No UberJob for qid=") + to_string(queryId) + " ujId=" + to_string(uberJobId) + " czar=" + to_string(czarId)); diff --git a/src/mysql/CMakeLists.txt b/src/mysql/CMakeLists.txt index 2796872566..23e418d931 100644 --- a/src/mysql/CMakeLists.txt +++ b/src/mysql/CMakeLists.txt @@ -12,6 +12,7 @@ target_sources(mysql PRIVATE target_link_libraries(mysql PUBLIC log mysqlclient_r + util ) install( @@ -22,6 +23,7 @@ add_executable(testLocalInfile testLocalInfile.cc) target_link_libraries(testLocalInfile mysql + util Boost::unit_test_framework ) diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index ae5a9f6b1d..7113b63ff9 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -260,11 +260,17 @@ CsvStream::CsvStream(std::size_t maxRecords) : _maxRecords(maxRecords) { } } +void CsvStream::cancel() { + std::unique_lock lock(_mtx); + _cancelled = true; + _cv.notify_all(); +} + void CsvStream::push(char const* data, std::size_t size) { std::unique_lock lock(_mtx); - while (_records.size() >= _maxRecords) { - _cv.wait(lock); - } + _cv.wait(lock, [this]() { return (_records.size() < _maxRecords) || _cancelled; }); + + if (_cancelled) return; if (data != nullptr && size != 0) { _records.emplace_back(std::make_shared(data, size)); } else { @@ -276,8 +282,19 @@ void CsvStream::push(char const* data, std::size_t size) { std::shared_ptr CsvStream::pop() { std::unique_lock lock(_mtx); - while (_records.empty()) { - _cv.wait(lock); + _cv.wait(lock, [this]() { return (!_records.empty() || _cancelled); }); + + if (_records.empty()) { + // _cancelled must be true. + // The hope is that this never happens, but to keep the system + // from locking up, send out illegal characters to force fail + // the merge. Need to keep sending characters until the + // database stops asking for them. + // See CsvStream::cancel() + _contaminated = true; + auto pstr = std::make_shared("$"); + _cv.notify_one(); + return pstr; } std::shared_ptr front = _records.front(); _records.pop_front(); diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index 4803a395d5..e9abcd113f 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -97,6 +97,17 @@ class CsvStream { */ void push(char const* data, std::size_t size); + /** + * Call to break push operations if the results are no longer needed. + * This is only meant to be used to break lingering push() calls. + * TODO:UJ The interleaving of result file reading and table + * merging makes it impossible to guarantee the result + * table is valid in the event that communication + * to a worker is lost during file transfer. + * @see UberJob::killUberJob() + */ + void cancel(); + /** * Pop a record from the stream. The method will block if the stream is empty * until a record is pushed. @@ -116,6 +127,12 @@ class CsvStream { void increaseBytesWrittenBy(size_t bytesToCopy) { _bytesWritten += bytesToCopy; } size_t getBytesWritten() const { return _bytesWritten; } + /** + * If this returns true, the result table has been contaminated by bad characters + * in an effort to keep the system from hanging, and the UserQuery is done. + */ + bool getContaminated() const { return _contaminated; } + private: CsvStream(std::size_t maxRecords); @@ -124,6 +141,8 @@ class CsvStream { std::size_t const _maxRecords; std::list> _records; std::atomic _bytesWritten; + bool _cancelled = false; + std::atomic _contaminated = false; }; /** diff --git a/src/qdisp/CzarStats.cc b/src/qdisp/CzarStats.cc index c0f19c5014..34061fe169 100644 --- a/src/qdisp/CzarStats.cc +++ b/src/qdisp/CzarStats.cc @@ -61,8 +61,8 @@ CzarStats::CzarStats(util::QdispPool::Ptr const& qdispPool) auto bucketValsRates = {128'000.0, 512'000.0, 1'024'000.0, 16'000'000.0, 128'000'000.0, 256'000'000.0, 512'000'000.0, 768'000'000.0, 1'000'000'000.0, 2'000'000'000.0, 4'000'000'000.0, 8'000'000'000.0}; - _histXRootDSSIRecvRate = util::HistogramRolling::Ptr( - new util::HistogramRolling("XRootDSSIRecvRateBytesPerSec", bucketValsRates, 1h, 10000)); + _histDataRecvRate = util::HistogramRolling::Ptr( + new util::HistogramRolling("DataRecvRateBytesPerSec", bucketValsRates, 1h, 10000)); _histMergeRate = util::HistogramRolling::Ptr( new util::HistogramRolling("MergeRateBytesPerSec", bucketValsRates, 1h, 10000)); _histFileReadRate = util::HistogramRolling::Ptr( @@ -102,10 +102,10 @@ void CzarStats::endQueryRespConcurrentProcessing(TIMEPOINT start, TIMEPOINT end) _histRespProcessing->addEntry(end, secs.count()); } -void CzarStats::addXRootDSSIRecvRate(double bytesPerSec) { - _histXRootDSSIRecvRate->addEntry(bytesPerSec); +void CzarStats::addDataRecvRate(double bytesPerSec) { + _histDataRecvRate->addEntry(bytesPerSec); LOGS(_log, LOG_LVL_TRACE, - "CzarStats::" << __func__ << " " << bytesPerSec << " " << _histXRootDSSIRecvRate->getString("")); + "CzarStats::" << __func__ << " " << bytesPerSec << " " << _histDataRecvRate->getString("")); } void CzarStats::addMergeRate(double bytesPerSec) { @@ -147,7 +147,7 @@ nlohmann::json CzarStats::getQdispStatsJson() const { nlohmann::json CzarStats::getTransmitStatsJson() const { nlohmann::json result; - result[_histXRootDSSIRecvRate->label()] = _histXRootDSSIRecvRate->getJson(); + result[_histDataRecvRate->label()] = _histDataRecvRate->getJson(); result[_histMergeRate->label()] = _histMergeRate->getJson(); result[_histFileReadRate->label()] = _histFileReadRate->getJson(); return result; diff --git a/src/qdisp/CzarStats.h b/src/qdisp/CzarStats.h index cddbd8b4e4..0e8bfb689b 100644 --- a/src/qdisp/CzarStats.h +++ b/src/qdisp/CzarStats.h @@ -76,7 +76,7 @@ class CzarStats : std::enable_shared_from_this { static Ptr get(); /// Add a bytes per second entry for query result transmits received over XRootD/SSI - void addXRootDSSIRecvRate(double bytesPerSec); + void addDataRecvRate(double bytesPerSec); /// Add a bytes per second entry for result merges void addMergeRate(double bytesPerSec); @@ -161,7 +161,7 @@ class CzarStats : std::enable_shared_from_this { uint64_t const _startTimeMs = 0; /// Histogram for tracking XROOTD/SSI receive rate in bytes per second. - util::HistogramRolling::Ptr _histXRootDSSIRecvRate; + util::HistogramRolling::Ptr _histDataRecvRate; /// Histogram for tracking merge rate in bytes per second. util::HistogramRolling::Ptr _histMergeRate; diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 9b47721961..57ef5e3161 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -333,6 +333,15 @@ void Executive::assignJobsToUberJobs() { void Executive::addMultiError(int errorCode, std::string const& errorMsg, int errorState) { util::Error err(errorCode, errorMsg, errorState); + + // Thousands of JOB_CANCEL errors are received and only the first one is of any value. + if (errorState == util::ErrorCode::JOB_CANCEL) { + if (++_jobCancelCount > 1) { + LOGS(_log, LOG_LVL_INFO, + " ignoring JOB_CANCEL already " << _jobCancelCount << " received " << errorMsg); + return; + } + } { lock_guard lock(_errorsMutex); _multiError.push_back(err); @@ -389,6 +398,7 @@ bool Executive::join() { cName(__func__) << " " << "Flag set to _empty=" << _empty << ", sCount=" << sCount << ", requestCount=" << _requestCount); + return _empty || isRowLimitComplete(); } diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 346cd7f6a5..969b56f866 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -294,9 +294,7 @@ class Executive : public std::enable_shared_from_this { mutable std::mutex _errorsMutex; std::condition_variable _allJobsComplete; - // TODO:UJ see what it takes to make this a normal mutex, before - // xrootd resulted in things being called in difficult to predict - // ways. That shouldn't be an issue any more. + // TODO:UJ see what it takes to make this a normal mutex. mutable std::recursive_mutex _jobMapMtx; QueryId _id = 0; ///< Unique identifier for this query. @@ -352,6 +350,7 @@ class Executive : public std::enable_shared_from_this { protojson::ScanInfo::Ptr _scanInfo; ///< Scan rating and tables. std::atomic _totalResultFileSize{0}; ///< Total size of all UberJob result files. + std::atomic _jobCancelCount{0}; ///< Total number of JOB_CANCEL messages received. }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 9b6faa6b32..86d19cb66e 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -63,7 +63,7 @@ bool JobQuery::cancel(bool superfluous) { VMUTEX_NOT_HELD(_jqMtx); lock_guard lock(_jqMtx); - string const context = _idStr + " cancel"; + string const context = _idStr + " job cancel"; LOGS(_log, LOG_LVL_DEBUG, context); auto exec = _executive.lock(); if (exec == nullptr) { @@ -71,7 +71,7 @@ bool JobQuery::cancel(bool superfluous) { return false; } if (!superfluous) { - exec->addMultiError(-1, context, util::ErrorCode::RESULT_IMPORT); + exec->addMultiError(-1, context, util::ErrorCode::JOB_CANCEL); } exec->markCompleted(getJobId(), false); return true; diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 2713e40a95..ad6fd11eba 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -85,6 +85,9 @@ class ResponseHandler { /// Signal an unrecoverable error condition. No further calls are expected. virtual void errorFlush(std::string const& msg, int code) = 0; + /// Stop an ongoing file merge, if possible. + virtual void cancelFileMerge() = 0; + /// Print a string representation of the receiver to an ostream virtual std::ostream& print(std::ostream& os) const = 0; diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index edfbcfce51..5413b97fa2 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -81,6 +81,12 @@ UberJob::UberJob(Executive::Ptr const& executive, std::shared_ptrcancelFileMerge(); +} + void UberJob::_setup() { UberJob::Ptr ujPtr = shared_from_this(); _respHandler->setUberJob(ujPtr); @@ -205,15 +211,6 @@ void UberJob::_unassignJobs() { _jobs.clear(); } -bool UberJob::isQueryCancelled() { - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " _executive == nullptr"); - return true; // Safer to assume the worst. - } - return exec->getCancelled(); -} - bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg) { // must be locked _jobsMtx auto currentState = _jobStatus->getState(); @@ -269,17 +266,17 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); - if (isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); - return _importResultError(true, "cancelled", "Query cancelled"); - } - auto exec = _executive.lock(); - if (exec == nullptr || exec->getCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive or cancelled"); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive"); return _importResultError(true, "cancelled", "Query cancelled - no executive"); } + if (exec->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); + return _importResultError(true, "cancelled", "Query cancelled"); + } + if (exec->isRowLimitComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { @@ -349,8 +346,8 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { bool const deleteData = true; bool const keepData = !deleteData; auto exec = _executive.lock(); - if (exec == nullptr || isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); + if (exec == nullptr || exec->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled " << errorMsg); return _workerErrorFinish(deleteData, "cancelled"); } @@ -364,6 +361,9 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { return _workerErrorFinish(keepData, "none", "rowLimitComplete"); } + string const eMsg = "host:" + _wContactInfo->getWHost() + " " + errorMsg; + exec->addMultiError(errorCode, eMsg, util::ErrorCode::WORKER_ERROR); + // Currently there are no detectable recoverable errors from workers. The only // error that a worker could send back that may possibly be recoverable would // be a missing table error, which is not trivial to detect. A worker local @@ -465,10 +465,11 @@ nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& e } void UberJob::killUberJob() { + // Usually called when a worker has effectively died. LOGS(_log, LOG_LVL_WARN, cName(__func__) << " stopping this UberJob and re-assigning jobs."); auto exec = _executive.lock(); - if (exec == nullptr || isQueryCancelled()) { + if (exec == nullptr || exec->getCancelled()) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); return; } @@ -491,6 +492,27 @@ void UberJob::killUberJob() { _unassignJobs(); // Let Czar::_monitor reassign jobs - other UberJobs are probably being killed // so waiting probably gets a better distribution. + + // If there are any ongoing file merges, they won't finish as the worker is dead. + // TODO:UJ - There is a chance this will ruin the result file, but it's either + // that or eventually hang the czar. The way worker result files are read + // and merged must be changed so the result file cannot be contaminated. + // Options are read everything in the file before merging + // - storing on disk, possibly slow + // - storing in memory limiting the number of concurrent transfers to + // avoid running out of memory + // - writing to partitions, the max number of partitions is a tiny + // fraction of the possible number of UberJobs, which will probably + // make failure recovery very complicated and slow. At minimum, all + // UberJobs writing to a partition should be running on the same + // worker. If a worker fails, all the jobs for that worker need + // to be sent to other workers. Making more partitions could break + // the limit on the number of partions. Attaching them to existing + // partitions runs the risk of another failure ruining a nearly + // complete partition (cascading failures). + // + getRespHandler()->cancelFileMerge(); + return; } diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index e1ef5f769b..313a4c7405 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -58,7 +58,7 @@ class UberJob : public std::enable_shared_from_this { UberJob(UberJob const&) = delete; UberJob& operator=(UberJob const&) = delete; - virtual ~UberJob() {}; + virtual ~UberJob(); std::string cName(const char* funcN) const { return std::string("UberJob::") + funcN + " " + getIdStr(); } @@ -75,7 +75,7 @@ class UberJob : public std::enable_shared_from_this { std::string const& getIdStr() const { return _idStr; } std::shared_ptr getRespHandler() { return _respHandler; } std::shared_ptr getStatus() { return _jobStatus; } - bool isQueryCancelled(); + void callMarkCompleteFunc(bool success); ///< call markComplete for all jobs in this UberJob. std::shared_ptr getExecutive() { return _executive.lock(); } @@ -155,7 +155,6 @@ class UberJob : public std::enable_shared_from_this { qmeta::JobStatus::Ptr _jobStatus{new qmeta::JobStatus()}; // TODO:UJ Maybe the JobStatus class should be // changed to better represent UberJobs - std::string _payload; ///< XrdSsi message to be sent to the _workerResource. TODO:UJ remove when possible std::weak_ptr _executive; std::shared_ptr _respHandler; QueryId const _queryId; diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 9840e0b2f1..9110048b74 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -92,6 +92,7 @@ class TestInfo : public ResponseHandler { } void flushHttpError(int errorCode, std::string const& errorMsg, int status) override {} void errorFlush(std::string const& msg, int code) override {}; + void cancelFileMerge() override {}; /// Print a string representation of the receiver to an ostream std::ostream& print(std::ostream& os) const override { diff --git a/src/qhttp/Server.cc b/src/qhttp/Server.cc index 24801bc350..6e167d1635 100644 --- a/src/qhttp/Server.cc +++ b/src/qhttp/Server.cc @@ -242,8 +242,11 @@ void Server::_readRequest(std::shared_ptr socket) { if (ec == asio::error::operation_aborted) { LOGLS_ERROR(_log, logger(self) << logger(socket) << "header read canceled"); } else if (ec) { - LOGLS_ERROR(_log, logger(self) - << logger(socket) << "header read failed: " << ec.message()); + // "End of file" happens very frequently and shouldn't be logged as an error. + auto logLvl = LOG_LVL_ERROR; + if (ec == asio::error::eof) logLvl = LOG_LVL_INFO; + LOGS(_log, logLvl, + logger(self) << logger(socket) << "header read failed: " << ec.message()); } timer->cancel(); if (ec) return; diff --git a/src/qmeta/JobStatus.h b/src/qmeta/JobStatus.h index 89ecda0c84..64f71c8c25 100644 --- a/src/qmeta/JobStatus.h +++ b/src/qmeta/JobStatus.h @@ -110,7 +110,7 @@ class JobStatus { // with each invocation of report(). State state; ///< Actual state TimeType stateTime; ///< Last modified timestamp - int stateCode; ///< Code associated with state (e.g. xrd or mysql error code) + int stateCode; ///< Code associated with state (e.g. mysql error code) std::string stateDesc; ///< Textual description std::string source = ""; ///< Source of the current state. MessageSeverity severity = MSG_INFO; ///< Severity of the message. diff --git a/src/qmeta/MessageStore.cc b/src/qmeta/MessageStore.cc index e4e32fe746..5dfbd9a7da 100644 --- a/src/qmeta/MessageStore.cc +++ b/src/qmeta/MessageStore.cc @@ -37,6 +37,8 @@ // Qserv headers #include "global/constants.h" +using namespace std; + namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qmeta.MessageStore"); } @@ -47,19 +49,25 @@ namespace lsst::qserv::qmeta { // public //////////////////////////////////////////////////////////////////////// +string QueryMessage::dump() const { + stringstream os; + os << "QueryMessage(chId=" << chunkId << " src=" << msgSource << " code=" << code + << " desc=" << description << " severity=" << severity << ")"; + return os.str(); +} + void MessageStore::addMessage(int chunkId, std::string const& msgSource, int code, std::string const& description, MessageSeverity severity, qmeta::JobStatus::TimeType timestamp) { if (timestamp == qmeta::JobStatus::TimeType()) { timestamp = qmeta::JobStatus::getNow(); } + QueryMessage qMsg(chunkId, msgSource, code, description, timestamp, severity); auto level = code < 0 ? LOG_LVL_ERROR : LOG_LVL_DEBUG; - LOGS(_log, level, "Add msg: " << chunkId << " " << msgSource << " " << code << " " << description); - { - std::lock_guard lock(_storeMutex); - _queryMessages.insert(_queryMessages.end(), - QueryMessage(chunkId, msgSource, code, description, timestamp, severity)); - } + LOGS(_log, level, "Add msg: " << qMsg.dump()); + + std::lock_guard lock(_storeMutex); + _queryMessages.push_back(qMsg); } void MessageStore::addErrorMessage(std::string const& msgSource, std::string const& description) { diff --git a/src/qmeta/MessageStore.h b/src/qmeta/MessageStore.h index 7fe9823c55..b5e72e8b06 100644 --- a/src/qmeta/MessageStore.h +++ b/src/qmeta/MessageStore.h @@ -60,6 +60,11 @@ struct QueryMessage { std::string description; qmeta::JobStatus::TimeType timestamp; MessageSeverity severity; + + /// Return string below the length to fit in source in database, which is varchar(63). + static std::string limitSrc(std::string const& src) { return src.substr(0, 62); } + + std::string dump() const; }; /** Store messages issued by Qserv workers and czar diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index e89c702c63..a1bf76119b 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -763,7 +763,9 @@ void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr cons for (auto const& elem : msgCountMap) { if (elem.second.count > _maxMsgSourceStore) { + // QMessages source column is VARCHAR(63) string source = string("MANY_") + elem.first; + source = QueryMessage::limitSrc(source); string desc = string("{\"msgSource\":") + elem.first + ", \"count\":" + to_string(elem.second.count) + "}"; qmeta::QueryMessage qm(-1, source, 0, desc, qmeta::JobStatus::getNow(), elem.second.severity); @@ -788,7 +790,8 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c (prevUpdateTime == chrono::time_point()) || (prevUpdateTime < updateTime); if (!force) { trans->commit(); - return QMetaChunkMap(); + chunkMap.updateTime = prevUpdateTime; + return chunkMap; } // Read the map itself @@ -910,11 +913,12 @@ void QMetaMysql::_addQueryMessage(QueryId queryId, qmeta::QueryMessage const& qM // build query std::string severity = (qMsg.severity == MSG_INFO ? "INFO" : "ERROR"); + string source = QueryMessage::limitSrc(qMsg.msgSource); string query = "INSERT INTO QMessages (queryId, msgSource, chunkId, code, severity, message, timestamp) VALUES " "("; query += to_string(queryId); - query += ", \"" + _conn->escapeString(qMsg.msgSource) + "\""; + query += ", \"" + _conn->escapeString(source) + "\""; query += ", " + to_string(qMsg.chunkId); query += ", " + to_string(qMsg.code); query += ", \"" + _conn->escapeString(severity) + "\""; diff --git a/src/replica/CMakeLists.txt b/src/replica/CMakeLists.txt index 8be3c55044..dab60c5ebc 100644 --- a/src/replica/CMakeLists.txt +++ b/src/replica/CMakeLists.txt @@ -25,8 +25,6 @@ target_link_libraries(replica PUBLIC replica_worker css global - XrdCl - XrdSsiLib http qhttp qmeta diff --git a/src/replica/apps/CMakeLists.txt b/src/replica/apps/CMakeLists.txt index f4cbae0dd3..da488ab41c 100644 --- a/src/replica/apps/CMakeLists.txt +++ b/src/replica/apps/CMakeLists.txt @@ -36,7 +36,6 @@ target_sources(replica_apps PRIVATE QhttpTestApp.cc QservWorkerApp.cc QservWorkerPingApp.cc - QservXrootdSsiApp.cc RebalanceApp.cc RegistryHttpApp.cc ReplicateApp.cc @@ -48,5 +47,4 @@ target_sources(replica_apps PRIVATE ) target_link_libraries(replica_apps PUBLIC log - XrdSsiLib ) diff --git a/src/replica/apps/QservXrootdSsiApp.cc b/src/replica/apps/QservXrootdSsiApp.cc deleted file mode 100644 index 13658dc2b1..0000000000 --- a/src/replica/apps/QservXrootdSsiApp.cc +++ /dev/null @@ -1,206 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// Class header -#include "replica/apps/QservXrootdSsiApp.h" - -// System headers -#include -#include -#include -#include -#include -#include -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiErrInfo.hh" -#include "XrdSsi/XrdSsiProvider.hh" -#include "XrdSsi/XrdSsiRequest.hh" -#include "XrdSsi/XrdSsiResource.hh" -#include "XrdSsi/XrdSsiService.hh" - -// Qserv headers -#include "global/ResourceUnit.h" -#include "util/BlockPost.h" -#include "util/TimeUtils.h" - -using namespace std; -using namespace lsst::qserv; - -extern XrdSsiProvider* XrdSsiProviderClient; - -namespace { -string const description = - "This application sends requests to Qserv workers over XROOTD/SSI for a purpose of testing" - " the performance, scalability and stability of the message delivery services."; - -bool const injectDatabaseOptions = false; -bool const boostProtobufVersionCheck = false; -bool const enableServiceProvider = false; - -double const millisecondsInSecond = 1000.; - -/// @return 'YYYY-MM-DD HH:MM:SS.mmm ' -string timestamp() { - return util::TimeUtils::toDateTimeString(chrono::milliseconds(util::TimeUtils::now())) + " "; -} - -string getErrorText(XrdSsiErrInfo const& e) { - ostringstream os; - int errCode; - os << "XrdSsiError error: " << e.Get(errCode); - os << ", code=" << errCode; - return os.str(); -} - -class SsiRequest : public XrdSsiRequest { -public: - explicit SsiRequest(string const& id, atomic& numFinishedRequests) - : _id(id), _numFinishedRequests(numFinishedRequests) {} - virtual ~SsiRequest() {} - char* GetRequest(int& requestLength) override { - // cout << "SsiRequest::" << __func__ << " id: " << _id << endl; - requestLength = 16; - return _requestData; - } - bool ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo const& rInfo) override { - int errCode; - eInfo.Get(errCode); - if (errCode != 0) { - // cout << "SsiRequest::" << __func__ << " id: " << _id << ": " << ::getErrorText(eInfo) << endl; - } - // Finished(); - //_numFinishedRequests.fetch_add(1); - return true; - } - void ProcessResponseData(XrdSsiErrInfo const& eInfo, char* buff, int blen, bool last) override { - int errCode; - eInfo.Get(errCode); - if (errCode != 0) { - cout << "SsiRequest::" << __func__ << " id: " << _id << ": " << ::getErrorText(eInfo) << endl; - } - } - -private: - string _id; - atomic& _numFinishedRequests; - char _requestData[1024]; -}; - -} // namespace - -namespace lsst::qserv::replica { - -QservXrootdSsiApp::Ptr QservXrootdSsiApp::create(int argc, char* argv[]) { - return Ptr(new QservXrootdSsiApp(argc, argv)); -} - -QservXrootdSsiApp::QservXrootdSsiApp(int argc, char* argv[]) - : Application(argc, argv, ::description, ::injectDatabaseOptions, ::boostProtobufVersionCheck, - ::enableServiceProvider) { - parser().required("url", "The connection URL for the XROOTD/SSI services.", _url) - .option("num-threads", "The number of threads for running the test.", _numThreads) - .option("report-interval-ms", - "An interval (milliseonds) for reporting the performance counters. Must be greater than " - "0.", - _reportIntervalMs) - .flag("progress", "The flag which would turn on periodic progress report on the requests.", - _progress) - .flag("verbose", "The flag which would turn on detailed report on the requests.", _verbose); -} - -int QservXrootdSsiApp::runImpl() { - // Connect to the service - XrdSsiErrInfo eInfo; - XrdSsiService* xrdSsiService = XrdSsiProviderClient->GetService(eInfo, _url); - if (!xrdSsiService) { - cerr << "Error obtaining XrdSsiService: serviceUrl=" << _url << ", " << ::getErrorText(eInfo) << endl; - return 1; - } - - // Counters updated by the requests - atomic numRequests(0); - atomic numFinishedRequests(0); - - // The requests - vector> requests; - mutex requestsMutex; - - // Launch all threads in the pool - atomic numThreadsActive{0}; - vector threads; - for (size_t i = 0; i < _numThreads; ++i) { - numThreadsActive.fetch_add(1); - threads.push_back(thread([&]() { - for (int chunk = 0; chunk < 150000; ++chunk) { - string const id = to_string(i) + ":" + to_string(chunk); - XrdSsiResource::Affinity const affinity = XrdSsiResource::Strong; - XrdSsiResource resource("/chk/wise_01/" + to_string(chunk), "", id, "", 0, affinity); - shared_ptr<::SsiRequest> request(new ::SsiRequest(id, numFinishedRequests)); - xrdSsiService->ProcessRequest(*(request.get()), resource); - ++numRequests; - lock_guard lock(requestsMutex); - requests.push_back(request); - } - // util::BlockPost bp(10*1000, 20*1000); - // bp.wait(); - numThreadsActive.fetch_sub(1); - cout << ::timestamp() << "Thread " << i << " finished" << endl; - })); - } - - // Begin the monitoring & reporting cycle - util::BlockPost bp(_reportIntervalMs, _reportIntervalMs + 1); - while (numThreadsActive.load() > 0) { - uint64_t beginNumRequests = numRequests; - bp.wait(_reportIntervalMs); - uint64_t const endNumRequests = numRequests; - double const requestsPerSecond = - (endNumRequests - beginNumRequests) / (_reportIntervalMs / millisecondsInSecond); - if (_progress) { - cout << ::timestamp() << "Sent: " << setprecision(7) << requestsPerSecond << " Req/s" << endl; - } - beginNumRequests = endNumRequests; - } - for (auto&& t : threads) { - t.join(); - } - // while (numFinishedRequests.load() < numRequests) { - // cout << ::timestamp() << "Waiting for all requests to finish: " << numFinishedRequests.load() << " - // / " - // << numRequests.load() << endl; - // bp.wait(1000); - // } - // cout << ::timestamp() << "All requests reported as fiished" << endl; - cout << ::timestamp() << "All threads finished, calling Finished() on " << requests.size() << " requests" - << endl; - for (auto&& request : requests) { - request->Finished(true); - } - cout << ::timestamp() << "Done calling Finished() on the requests" << endl; - util::BlockPost bp1(10 * 1000, 20 * 1000); - bp1.wait(); - return 0; -} - -} // namespace lsst::qserv::replica diff --git a/src/replica/apps/QservXrootdSsiApp.h b/src/replica/apps/QservXrootdSsiApp.h deleted file mode 100644 index 8a990bf2d5..0000000000 --- a/src/replica/apps/QservXrootdSsiApp.h +++ /dev/null @@ -1,74 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_REPLICA_QSERVXROOTDSSIAPP_H -#define LSST_QSERV_REPLICA_QSERVXROOTDSSIAPP_H - -// System headers -#include -#include - -// Qserv headers -#include "replica/apps/Application.h" - -// This header declarations -namespace lsst::qserv::replica { - -/** - * Class QservXrootdSsiApp is performance and scalability test for the XROOTD/SSI. - */ -class QservXrootdSsiApp : public Application { -public: - /// The pointer type for instances of the class - typedef std::shared_ptr Ptr; - - /** - * The factory method is the only way of creating objects of this class - * because of the very base class's inheritance from 'enable_shared_from_this'. - * - * @param argc The number of command-line arguments. - * @param argv The vector of command-line arguments. - */ - static Ptr create(int argc, char* argv[]); - - QservXrootdSsiApp() = delete; - QservXrootdSsiApp(QservXrootdSsiApp const&) = delete; - QservXrootdSsiApp& operator=(QservXrootdSsiApp const&) = delete; - - virtual ~QservXrootdSsiApp() final = default; - -protected: - /// @see Application::runImpl() - virtual int runImpl() final; - -private: - /// @see QservXrootdSsiApp::create() - QservXrootdSsiApp(int argc, char* argv[]); - - std::string _url; ///< The connection URL for the XROOTD/SSI services. - size_t _numThreads = 1; ///< The number of threads for running the test. - int _reportIntervalMs = 1000; ///< An interval for reporting the performance counters. - bool _progress = false; ///< For periodic progress reports on the requests. - bool _verbose = false; ///< For detailed report on the requests. -}; - -} // namespace lsst::qserv::replica - -#endif /* LSST_QSERV_REPLICA_QSERVXROOTDSSIAPP_H */ diff --git a/src/replica/qserv/CMakeLists.txt b/src/replica/qserv/CMakeLists.txt index 3485c46a0c..0ee06c6263 100644 --- a/src/replica/qserv/CMakeLists.txt +++ b/src/replica/qserv/CMakeLists.txt @@ -19,7 +19,6 @@ target_sources(replica_qserv PRIVATE RemoveReplicaQservMgtRequest.cc SetReplicasQservMgtRequest.cc TestEchoQservMgtRequest.cc - XrdCmsgetVnId.cc ) target_link_libraries(replica_qserv PUBLIC log diff --git a/src/replica/qserv/QservMgtServices.h b/src/replica/qserv/QservMgtServices.h index 7256c86238..09dc06c1e1 100644 --- a/src/replica/qserv/QservMgtServices.h +++ b/src/replica/qserv/QservMgtServices.h @@ -44,8 +44,6 @@ #include "wbase/TaskState.h" // Forward declarations -class XrdSsiService; - namespace lsst::qserv::wbase { class TaskSelector; } // namespace lsst::qserv::wbase diff --git a/src/replica/qserv/XrdCmsgetVnId.cc b/src/replica/qserv/XrdCmsgetVnId.cc deleted file mode 100644 index 4173cbe8dc..0000000000 --- a/src/replica/qserv/XrdCmsgetVnId.cc +++ /dev/null @@ -1,110 +0,0 @@ -/* - * LSST Data Management System - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ - -// System headers -#include -#include -#include - -// Qserv headers -#include "global/stringUtil.h" -#include "replica/config/Configuration.h" -#include "replica/mysql/DatabaseMySQL.h" -#include "replica/mysql/DatabaseMySQLUtils.h" -#include "util/String.h" - -// XrootD headers -#include "XrdCms/XrdCmsVnId.hh" -#include "XrdSys/XrdSysError.hh" -#include "XrdVersion.hh" - -using namespace std; -using namespace lsst::qserv::replica; -using namespace lsst::qserv::replica::database::mysql; -namespace util = lsst::qserv::util; - -/** - * @brief Read a value of the VNID from the Qserv worker database that's - * configured via a MySQL connection string passed among the input - * parameters of the function. - * - * The list of input parameters has the following syntax: - * @code - * - * @code - * - * Where: - * work-db-conn-url: the database connector string for the worker's MySQL service - * max-reconnects: the maximum number of reconnects to he service - * conn-timeout-sec: the timeout for connecting to the service and executing the query - */ -extern "C" string XrdCmsgetVnId(XrdCmsgetVnIdArgs) { - string const context = string(__func__) + ": "; - string vnId; - try { - bool const greedy = true; - vector args = util::String::split(parms, " ", greedy); - if (args.size() != 3) { - eDest.Say(context.data(), "illegal number of parameters for the plugin. ", - "Exactly 3 parameters are required: " - "."); - } else { - string const qservWorkerDbUrl = args[0]; - Configuration::setQservWorkerDbUrl(qservWorkerDbUrl); - // Parameter 'maxReconnects' limits the total number of retries to execute the query in case - // if the query fails during execution. If the parameter's value is set to 0 then the default - // value of the parameter will be pulled by the query processor from the Replication - // system's Configuration. - unsigned int maxReconnects = lsst::qserv::stoui(args[1]); - // Parameter 'timeoutSec' is used both while connecting to the database server and for executing - // the query. If the MySQl service won't respond to the connection attempts beyond a period of - // time specified by the parameter then the operation will fail. Similarly, if the query execution - // will take longer than it's specified in the parameter then the query will fail. If the - // parameter's value is set to 0 then the default value of the parameter will be pulled by the - // query processor from the Replication system's Configuration. - unsigned int timeoutSec = lsst::qserv::stoui(args[2]); - // This parameter allows the database connector to make reconnects if the MySQL service - // won't be responding (or not be up) at the initial connection attempt. - bool const allowReconnects = true; - // Using the RAII-style connection handler to automatically close the connection and - // release resources in case of exceptions. - ConnectionHandler const handler(Connection::open2( - Configuration::qservWorkerDbParams("qservw_worker"), allowReconnects, timeoutSec)); - QueryGenerator const g(handler.conn); - handler.conn->executeInOwnTransaction( - [&context, &vnId, &eDest, &g](decltype(handler.conn) conn) { - string const query = g.select("id") + g.from("Id"); - if (!selectSingleValue(conn, query, vnId)) { - eDest.Say(context.data(), - "worker identity is not set in the Qserv worker database."); - } - }, - maxReconnects, timeoutSec); - } - } catch (exception const& ex) { - eDest.Say(context.data(), - "failed to pull worker identity from the Qserv worker database, ex:", ex.what()); - } - eDest.Say(context.data(), "vnid: ", vnId.data()); - return vnId; -} - -XrdVERSIONINFO(XrdCmsgetVnId, vnid_mysql_0); diff --git a/src/replica/tools/qserv-replica-test.cc b/src/replica/tools/qserv-replica-test.cc index 1dec5cb5d9..6d7136cc8c 100644 --- a/src/replica/tools/qserv-replica-test.cc +++ b/src/replica/tools/qserv-replica-test.cc @@ -34,7 +34,6 @@ #include "replica/apps/QhttpTestApp.h" #include "replica/apps/TransactionsApp.h" #include "replica/apps/QservWorkerPingApp.h" -#include "replica/apps/QservXrootdSsiApp.h" using namespace std; using namespace lsst::qserv::replica; @@ -52,7 +51,6 @@ ApplicationColl getAppColl() { coll.add("QHTTP"); coll.add("TRANSACTIONS"); coll.add("WORKER-PING"); - coll.add("XROOTD-SSI"); return coll; } } // namespace diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 9b72bc9e69..d05ad55cce 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -171,15 +171,6 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); } - // Do nothing if the query got cancelled for any reason. - if (uberJob->isQueryCancelled()) { - return true; - } - auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { - return true; - } - // Check if the final result size is too large. It should be safe to do this // here as the only expected errors at this point are failures in transmission. // Even if there is a failure in transmission, the retry would be expected @@ -189,12 +180,11 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi _perJobResultSize[uJobId] += fileSize; size_t tResultSize = _totalResultSize + _perJobResultSize[uJobId]; if (tResultSize > _maxResultTableSizeBytes) { - ostringstream os; - os << queryIdJobStr << " cancelling the query, queryResult table " << _mergeTable - << " is too large at " << tResultSize << " bytes, max allowed size is " - << _maxResultTableSizeBytes << " bytes"; - LOGS(_log, LOG_LVL_ERROR, os.str()); - _error = util::Error(-1, os.str(), -1); + string str = queryIdJobStr + " cancelling the query, queryResult table " + _mergeTable + + " is too large at " + to_string(tResultSize) + " bytes, max allowed size is " + + to_string(_maxResultTableSizeBytes) + " bytes"; + LOGS(_log, LOG_LVL_ERROR, str); + _error = util::Error(-1, str, -1); return false; } } @@ -203,7 +193,7 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi bool success) { if (!success) return; if (std::chrono::duration const seconds = end - start; seconds.count() > 0) { - qdisp::CzarStats::get()->addXRootDSSIRecvRate(bytes / seconds.count()); + qdisp::CzarStats::get()->addDataRecvRate(bytes / seconds.count()); } }; auto tct = make_shared>(cbf); @@ -227,6 +217,13 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi return true; } + // Don't merge if the query got cancelled. + auto executive = uberJob->getExecutive(); + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { + csvStream->cancel(); // After this point, the file has to be read + return true; + } + auto start = std::chrono::system_clock::now(); ret = _applyMysqlMyIsam(infileStatement, fileSize); auto end = std::chrono::system_clock::now(); diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index f86ea5db95..dd7e0be1ec 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -49,7 +49,6 @@ namespace mysql { class CsvStream; class MysqlConfig; } // namespace mysql - namespace qdisp { class MessageStore; class UberJob; diff --git a/src/sql/SqlResults.cc b/src/sql/SqlResults.cc index 36b89e085a..1ccd3a56fb 100644 --- a/src/sql/SqlResults.cc +++ b/src/sql/SqlResults.cc @@ -33,6 +33,9 @@ // LSST headers #include "lsst/log/Log.h" +// LSST headers +#include "lsst/log/Log.h" + // Qserv headers #include "mysql/SchemaFactory.h" diff --git a/src/sql/SqlResults.h b/src/sql/SqlResults.h index f5894b4c8d..5c3a115410 100644 --- a/src/sql/SqlResults.h +++ b/src/sql/SqlResults.h @@ -32,6 +32,7 @@ // System headers #include #include +#include #include #include @@ -104,6 +105,8 @@ class SqlResults : boost::noncopyable { // TODO:UJ for most of these functions, calling extractFirstXColumns // directly may make more sense than calling extractFirst6Columns. // Not changing this now as it will make rebasing difficult. + // TODO:UJ - There may be a better way to do this with std::reference_wrapper + // variadic function templates. bool extractFirstXColumns(std::vector*> const& vectorRef, SqlErrorObject& sqlErr); bool extractFirstColumn(std::vector& col1, SqlErrorObject& errObj); diff --git a/src/util/ConfigStore.h b/src/util/ConfigStore.h index 7397b2589e..63bf5d1b96 100644 --- a/src/util/ConfigStore.h +++ b/src/util/ConfigStore.h @@ -27,7 +27,7 @@ * * @brief Provide common configuration management framework * - * Manage czar and worker (xrdssi plugin) configuration files + * Manage czar and worker configuration files * * @author Fabrice Jammes, IN2P3/SLAC */ diff --git a/src/util/Error.h b/src/util/Error.h index 8f303ee088..676eb70f3f 100644 --- a/src/util/Error.h +++ b/src/util/Error.h @@ -62,8 +62,10 @@ struct ErrorCode { MYSQLEXEC, INTERNAL, CZAR_RESULT_TOO_LARGE, + JOB_CANCEL, // Worker errors: - WORKER_RESULT_TOO_LARGE + WORKER_RESULT_TOO_LARGE, + WORKER_ERROR }; }; diff --git a/src/util/FileMonitor.cc b/src/util/FileMonitor.cc index b7d5e5acc6..015ec9b0d9 100644 --- a/src/util/FileMonitor.cc +++ b/src/util/FileMonitor.cc @@ -69,9 +69,8 @@ void FileMonitor::_checkLoop() { char buffer[EVENT_BUF_LEN]; /// There's a lock situation here. If the file is never modified, it's never getting past - /// this line. xrootd doesn't exit gracefully anyway, so this is unlikely to cause a problem. - /// This thread could be cancelled or the file could be touched, but that's unlikely to make - /// program termination much prettier. + /// this line. qserv doesn't exit gracefully anyway, so this is unlikely to cause a problem. + /// This thread could be cancelled or the file could be touched. int length = read(_fD, buffer, EVENT_BUF_LEN); LOGS(_log, LOG_LVL_WARN, "FileMonitor::checkLoop() " << _fileName << " read length=" << length); if (length < 0) { diff --git a/src/util/QdispPool.h b/src/util/QdispPool.h index 562450624d..6dc170599a 100644 --- a/src/util/QdispPool.h +++ b/src/util/QdispPool.h @@ -140,11 +140,9 @@ class PriorityQueue : public util::CommandQueue { }; /// This class is used to provide a pool of threads for handling out going -/// and incoming messages from xrootd as well as a system for prioritizing +/// and incoming messages as well as a system for prioritizing /// the messages. -/// This has not worked entirely as intended. Reducing the number of threads -/// had negative impacts on xrootd, but other changes have been made such that -/// reducing the size of the thread pools can be tried again. +/// This has not worked entirely as intended. /// What it does do is prioritize outgoing messages (typically jobs going to /// workers), allow interactive queries to be handled quickly, even under /// substantial loads, and it gives a good idea of how busy the czar really diff --git a/src/wbase/Base.h b/src/wbase/Base.h index 8dc1b5662a..74cdada0cc 100644 --- a/src/wbase/Base.h +++ b/src/wbase/Base.h @@ -34,13 +34,10 @@ #include "boost/format.hpp" // Forward declarations -class XrdSysError; -class XrdSysLogger; -class XrdSfsAio; -namespace lsst::qserv { namespace wbase { +namespace lsst::qserv::wbase { class StringBuffer; class StringBuffer2; -}} // namespace lsst::qserv::wbase +} // namespace lsst::qserv::wbase namespace lsst::qserv::wbase { diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index f0ae325c28..a930c4b565 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -234,7 +234,7 @@ class FileChannelShared { bool _isUberJob; ///< true if this is using UberJob http. To be removed when _sendChannel goes away. - std::shared_ptr const _sendChannel; ///< Used to pass encoded information to XrdSsi. + std::shared_ptr const _sendChannel; ///< Used to send info to czar. std::weak_ptr _uberJobData; ///< Contains czar contact info. UberJobId const _uberJobId; ///< The UberJobId diff --git a/src/wbase/SendChannel.cc b/src/wbase/SendChannel.cc index f8a5485cda..7cc0c2cc80 100644 --- a/src/wbase/SendChannel.cc +++ b/src/wbase/SendChannel.cc @@ -50,7 +50,7 @@ using namespace std; namespace lsst::qserv::wbase { /// NopChannel is a NOP implementation of SendChannel for development and -/// debugging code without an XrdSsi channel. +/// debugging code without an actual channel. class NopChannel : public SendChannel { public: NopChannel() {} diff --git a/src/wbase/SendChannel.h b/src/wbase/SendChannel.h index 56f2a598c8..bfa3a2d174 100644 --- a/src/wbase/SendChannel.h +++ b/src/wbase/SendChannel.h @@ -30,9 +30,9 @@ namespace lsst::qserv { namespace wbase { -/// SendChannel objects abstract an byte-output mechanism. Provides a layer of -/// abstraction to reduce coupling to the XrdSsi API. SendChannel generally -/// accepts only one call to send bytes, unless the sendStream call is used. +/// SendChannel is used to send information about results +/// and errors back to the czar so that the czar can collect +/// the results or cancel the related data. class SendChannel { public: using Ptr = std::shared_ptr; diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 0f3a249a2f..66eabf8ed2 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -149,7 +149,6 @@ class Task : public util::CommandForThreadPool { // - hasSubchunks seems redundant. // Hopefully, many are the same for all tasks and can be moved to ujData and userQueryInfo. // Candidates: maxTableSizeMb, FileChannelShared, resultsHttpPort. - // Unfortunately, this will be much easier if it is done after xrootd method is removed. Task(std::shared_ptr const& ujData, int jobId, int attemptCount, int chunkId, int fragmentNumber, size_t templateId, bool hasSubchunks, int subchunkId, std::string const& db, std::vector const& fragSubTables, std::vector const& fragSubchunkIds, diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 289ea72e6c..e4a59e3d20 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -194,8 +194,7 @@ void UberJobData::_queueUJResponse(http::Method method_, std::vector @@ -38,7 +38,7 @@ using namespace std; -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { HttpModule::HttpModule(string const& context, shared_ptr const& foreman, shared_ptr const& req, shared_ptr const& resp) @@ -94,4 +94,4 @@ wbase::TaskSelector HttpModule::translateTaskSelector(string const& func) const return selector; } -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms diff --git a/src/xrdsvc/HttpModule.h b/src/wcomms/HttpModule.h similarity index 93% rename from src/xrdsvc/HttpModule.h rename to src/wcomms/HttpModule.h index fb122041d6..e8cfd7083b 100644 --- a/src/xrdsvc/HttpModule.h +++ b/src/wcomms/HttpModule.h @@ -18,8 +18,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_XRDSVC_HTTPMODULE_H -#define LSST_QSERV_XRDSVC_HTTPMODULE_H +#ifndef LSST_QSERV_WCOMMS_HTTPMODULE_H +#define LSST_QSERV_WCOMMS_HTTPMODULE_H // System headers #include @@ -43,7 +43,7 @@ class Foreman; } // namespace lsst::qserv::wcontrol // This header declarations -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { /** * Class HttpModule is an intermediate base class of the Qserv worker modules. @@ -85,6 +85,6 @@ class HttpModule : public http::QhttpModule { std::shared_ptr const _foreman; }; -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms -#endif // LSST_QSERV_XRDSVC_HTTPMODULE_H +#endif // LSST_QSERV_WCOMMS_HTTPMODULE_H diff --git a/src/xrdsvc/HttpMonitorModule.cc b/src/wcomms/HttpMonitorModule.cc similarity index 97% rename from src/xrdsvc/HttpMonitorModule.cc rename to src/wcomms/HttpMonitorModule.cc index bbd4c65667..9210defb8c 100644 --- a/src/xrdsvc/HttpMonitorModule.cc +++ b/src/wcomms/HttpMonitorModule.cc @@ -20,7 +20,7 @@ */ // Class header -#include "xrdsvc/HttpMonitorModule.h" +#include "wcomms/HttpMonitorModule.h" // System headers #include @@ -41,7 +41,7 @@ using namespace std; using json = nlohmann::json; -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { void HttpMonitorModule::process(string const& context, shared_ptr const& foreman, shared_ptr const& req, @@ -133,4 +133,4 @@ json HttpMonitorModule::_echo() { return json::object({{"data", body().required("data")}}); } -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms diff --git a/src/xrdsvc/HttpMonitorModule.h b/src/wcomms/HttpMonitorModule.h similarity index 91% rename from src/xrdsvc/HttpMonitorModule.h rename to src/wcomms/HttpMonitorModule.h index 01c5c171c5..fb9d1ab573 100644 --- a/src/xrdsvc/HttpMonitorModule.h +++ b/src/wcomms/HttpMonitorModule.h @@ -18,8 +18,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_XRDSVC_HTTPMONITORMODULE_H -#define LSST_QSERV_XRDSVC_HTTPMONITORMODULE_H +#ifndef LSST_QSERV_WCOMMS_HTTPMONITORMODULE_H +#define LSST_QSERV_WCOMMS_HTTPMONITORMODULE_H // System headers #include @@ -29,7 +29,7 @@ #include "nlohmann/json.hpp" // Qserv headers -#include "xrdsvc/HttpModule.h" +#include "wcomms/HttpModule.h" namespace lsst::qserv::qhttp { class Request; @@ -41,13 +41,13 @@ class Foreman; } // namespace lsst::qserv::wcontrol // This header declarations -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { /** * Class HttpMonitorModule implements a handler for reporting various run-time * monitoring metrics and statistics collected at the Qserv worker. */ -class HttpMonitorModule : public xrdsvc::HttpModule { +class HttpMonitorModule : public wcomms::HttpModule { public: /** * @note supported values for parameter 'subModuleName' are: @@ -94,6 +94,6 @@ class HttpMonitorModule : public xrdsvc::HttpModule { nlohmann::json _echo(); }; -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms -#endif // LSST_QSERV_XRDSVC_HTTPMONITORMODULE_H +#endif // LSST_QSERV_WCOMMS_HTTPMONITORMODULE_H diff --git a/src/xrdsvc/HttpReplicaMgtModule.cc b/src/wcomms/HttpReplicaMgtModule.cc similarity index 94% rename from src/xrdsvc/HttpReplicaMgtModule.cc rename to src/wcomms/HttpReplicaMgtModule.cc index 91692aa921..4fb4940beb 100644 --- a/src/xrdsvc/HttpReplicaMgtModule.cc +++ b/src/wcomms/HttpReplicaMgtModule.cc @@ -20,7 +20,7 @@ */ // Class header -#include "xrdsvc/HttpReplicaMgtModule.h" +#include "wcomms/HttpReplicaMgtModule.h" // System headers #include @@ -29,7 +29,6 @@ // Third party headers #include "lsst/log/Log.h" -#include "XrdSsi/XrdSsiCluster.hh" // Qserv headers #include "http/Exceptions.h" @@ -40,17 +39,14 @@ #include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" #include "wcontrol/ResourceMonitor.h" +#include "wmain/WorkerMain.h" #include "wpublish/ChunkInventory.h" -#include "xrdsvc/SsiProvider.h" -#include "xrdsvc/XrdName.h" - -extern XrdSsiProvider* XrdSsiProviderLookup; using namespace std; using json = nlohmann::json; namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.HttpReplicaMgt"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.wcomms.HttpReplicaMgt"); } namespace { @@ -64,7 +60,7 @@ string makeResource(string const& database, int chunk) { return "/chk/" + databa } // namespace -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { void HttpReplicaMgtModule::process(string const& context, shared_ptr const& foreman, shared_ptr const& req, @@ -186,8 +182,7 @@ json HttpReplicaMgtModule::_rebuildInventory() { // Load the persistent inventory data into the transient one. wpublish::ChunkInventory newChunkInventory; try { - xrdsvc::XrdName x; - newChunkInventory.init(x.getName(), foreman()->mySqlConfig()); + newChunkInventory.init(wmain::WorkerMain::get()->getName(), foreman()->mySqlConfig()); } catch (exception const& ex) { throw http::Error(__func__, "persistent inventory read failed, ex: " + string(ex.what())); } @@ -216,8 +211,7 @@ json HttpReplicaMgtModule::_rebuildInventory() { void HttpReplicaMgtModule::_rebuildPersistentInventory() const { wpublish::ChunkInventory newChunkInventory; try { - xrdsvc::XrdName x; - newChunkInventory.rebuild(x.getName(), foreman()->mySqlConfig()); + newChunkInventory.rebuild(wmain::WorkerMain::get()->getName(), foreman()->mySqlConfig()); } catch (exception const& ex) { throw http::Error(__func__, "inventory rebuild stage failed, ex: " + string(ex.what())); } @@ -256,7 +250,7 @@ void HttpReplicaMgtModule::_updateInventory(string const& func, } } - // Update the current map and notify XRootD accordingly. + // Update the current map. for (auto&& [database, chunks] : toBeRemovedExistMap) { if (databaseFilter.contains(database)) { for (int const chunk : chunks) { @@ -327,9 +321,7 @@ void HttpReplicaMgtModule::_modifyChunk(string const& func, int chunk, string co string const resource = ::makeResource(database, chunk); debug(func, operation + " resource: " + resource + ", DataContext: " + to_string(_dataContext)); try { - // Notify XRootD/cmsd and (depending on a mode) modify the provider's - // copy of the inventory. After that modify both (persistent and - // transient) inventories. + // Modify both (persistent and transient) inventories. if (Direction::ADD == direction) { foreman()->chunkInventory()->add(database, chunk, foreman()->mySqlConfig()); } else { @@ -344,4 +336,4 @@ void HttpReplicaMgtModule::_modifyChunk(string const& func, int chunk, string co } } -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms diff --git a/src/xrdsvc/HttpReplicaMgtModule.h b/src/wcomms/HttpReplicaMgtModule.h similarity index 94% rename from src/xrdsvc/HttpReplicaMgtModule.h rename to src/wcomms/HttpReplicaMgtModule.h index ac58a58283..a5e3af5bbd 100644 --- a/src/xrdsvc/HttpReplicaMgtModule.h +++ b/src/wcomms/HttpReplicaMgtModule.h @@ -18,8 +18,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_XRDSVC_HTTPREPLICAMGTMODULE_H -#define LSST_QSERV_XRDSVC_HTTPREPLICAMGTMODULE_H +#ifndef LSST_QSERV_WCOMMS_HTTPREPLICAMGTMODULE_H +#define LSST_QSERV_WCOMMS_HTTPREPLICAMGTMODULE_H // System headers #include @@ -31,11 +31,9 @@ #include "nlohmann/json.hpp" // Qserv headers -#include "xrdsvc/HttpModule.h" +#include "wcomms/HttpModule.h" // Forward declarations -class XrdSsiCluster; - namespace lsst::qserv::qhttp { class Request; class Response; @@ -49,12 +47,8 @@ namespace lsst::qserv::wpublish { class ChunkInventory; } // namespace lsst::qserv::wpublish -namespace lsst::qserv::xrdsvc { -class SsiProviderServer; -} // namespace lsst::qserv::xrdsvc - // This header declarations -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { /** * Class HttpReplicaMgtModule implements a handler for managing chunk replicas @@ -62,7 +56,7 @@ namespace lsst::qserv::xrdsvc { * databases that were partitioned based on the same partitioning configuration) * or all known databases. */ -class HttpReplicaMgtModule : public xrdsvc::HttpModule { +class HttpReplicaMgtModule : public wcomms::HttpModule { public: /** * @note supported values for parameter 'subModuleName' are: @@ -187,6 +181,6 @@ class HttpReplicaMgtModule : public xrdsvc::HttpModule { bool _dataContext = false; }; -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms -#endif // LSST_QSERV_XRDSVC_HTTPREPLICAMGTMODULE_H +#endif // LSST_QSERV_WCOMMS_HTTPREPLICAMGTMODULE_H diff --git a/src/xrdsvc/HttpSvc.cc b/src/wcomms/HttpSvc.cc similarity index 95% rename from src/xrdsvc/HttpSvc.cc rename to src/wcomms/HttpSvc.cc index 9694f0533e..6a1e005bef 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/wcomms/HttpSvc.cc @@ -20,7 +20,7 @@ */ // Class header -#include "xrdsvc/HttpSvc.h" +#include "wcomms/HttpSvc.h" // System headers #include @@ -31,9 +31,9 @@ #include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" #include "wpublish/ChunkInventory.h" -#include "xrdsvc/HttpMonitorModule.h" -#include "xrdsvc/HttpReplicaMgtModule.h" -#include "xrdsvc/HttpWorkerCzarModule.h" +#include "wcomms/HttpMonitorModule.h" +#include "wcomms/HttpReplicaMgtModule.h" +#include "wcomms/HttpWorkerCzarModule.h" // LSST headers #include "lsst/log/Log.h" @@ -43,13 +43,13 @@ using namespace std; namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.HttpSvc"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.wcomms.HttpSvc"); string const serviceName = "WORKER-MANAGEMENT "; } // namespace -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { shared_ptr HttpSvc::create(shared_ptr const& foreman, uint16_t port, unsigned int numThreads) { @@ -60,7 +60,7 @@ HttpSvc::HttpSvc(shared_ptr const& foreman, uint16_t port, un : _foreman(foreman), _port(port), _numThreads(numThreads) {} uint16_t HttpSvc::start() { - string const context = "xrdsvc::HttpSvc::" + string(__func__) + " "; + string const context = "wcomms::HttpSvc::" + string(__func__) + " "; lock_guard const lock(_mtx); if (_httpServerPtr != nullptr) { throw logic_error(context + "the service is already running."); @@ -167,7 +167,7 @@ uint16_t HttpSvc::start() { } void HttpSvc::stop() { - string const context = "xrdsvc::HttpSvc::" + string(__func__) + " "; + string const context = "wcomms::HttpSvc::" + string(__func__) + " "; lock_guard const lock(_mtx); if (_httpServerPtr == nullptr) { throw logic_error(context + "the service is not running."); @@ -185,4 +185,4 @@ void HttpSvc::stop() { LOGS(_log, LOG_LVL_INFO, context + "stopped"); } -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms diff --git a/src/xrdsvc/HttpSvc.h b/src/wcomms/HttpSvc.h similarity index 95% rename from src/xrdsvc/HttpSvc.h rename to src/wcomms/HttpSvc.h index a7796e3637..bf8e14c294 100644 --- a/src/xrdsvc/HttpSvc.h +++ b/src/wcomms/HttpSvc.h @@ -18,8 +18,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_XRDSVC_HTTPSVC_H -#define LSST_QSERV_XRDSVC_HTTPSVC_H +#ifndef LSST_QSERV_WCOMMS_HTTPSVC_H +#define LSST_QSERV_WCOMMS_HTTPSVC_H // System headers #include @@ -40,7 +40,7 @@ class Foreman; } // namespace lsst::qserv::wcontrol // This header declarations -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { /** * Class HttpSvc is the HTTP server for processing worker management requests. @@ -54,7 +54,7 @@ namespace lsst::qserv::xrdsvc { * // Create the server. Note, it won't run yet until explicitly started. * uint16_t const port = 0; // The port will be dynamically allocated at start * unsigned int const numThreads = 2; // The number of BOOST ASIO threads - * auto const svc = xrdsvc::HttpSvc::create(port, numThreads); + * auto const svc = wcomms::HttpSvc::create(port, numThreads); * * // Start the server and get the actual port number. * uint16_t const actualPort = svc->start(); @@ -135,6 +135,6 @@ class HttpSvc : public std::enable_shared_from_this { std::vector> _threads; }; -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms -#endif // LSST_QSERV_XRDSVC_HTTPSVC_H +#endif // LSST_QSERV_WCOMMS_HTTPSVC_H diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/wcomms/HttpWorkerCzarModule.cc similarity index 98% rename from src/xrdsvc/HttpWorkerCzarModule.cc rename to src/wcomms/HttpWorkerCzarModule.cc index b9446ec62d..8a8ebd02f6 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/wcomms/HttpWorkerCzarModule.cc @@ -20,7 +20,7 @@ */ // Class header -#include "xrdsvc/HttpWorkerCzarModule.h" +#include "wcomms/HttpWorkerCzarModule.h" // System headers #include @@ -57,14 +57,12 @@ #include "wpublish/QueriesAndChunks.h" #include "wpublish/QueryStatistics.h" #include "wsched/BlendScheduler.h" -#include "xrdsvc/SsiProvider.h" -#include "xrdsvc/XrdName.h" using namespace std; using json = nlohmann::json; namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.HttpReplicaMgt"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.wcomms.HttpReplicaMgt"); } namespace { @@ -77,7 +75,7 @@ json const extErrorReplicaInUse = json::object({{"in_use", 1}}); } // namespace -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { void HttpWorkerCzarModule::process(string const& context, shared_ptr const& foreman, shared_ptr const& req, @@ -333,4 +331,4 @@ json HttpWorkerCzarModule::_handleQueryStatus(std::string const& func) { return jsRet; } -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/wcomms/HttpWorkerCzarModule.h similarity index 90% rename from src/xrdsvc/HttpWorkerCzarModule.h rename to src/wcomms/HttpWorkerCzarModule.h index 00a6d4fd67..5ea2b956a3 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.h +++ b/src/wcomms/HttpWorkerCzarModule.h @@ -18,8 +18,8 @@ * the GNU General Public License along with this program. If not, * see . */ -#ifndef LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H -#define LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H +#ifndef LSST_QSERV_WCOMMS_HTTPWORKERCZARMODULE_H +#define LSST_QSERV_WCOMMS_HTTPWORKERCZARMODULE_H // System headers #include @@ -32,7 +32,7 @@ // Qserv headers #include "qmeta/types.h" -#include "xrdsvc/HttpModule.h" +#include "wcomms/HttpModule.h" namespace lsst::qserv::protojson { class CzarContactInfo; @@ -53,15 +53,11 @@ namespace lsst::qserv::wcontrol { class Foreman; } // namespace lsst::qserv::wcontrol -namespace lsst::qserv::xrdsvc { -class SsiProviderServer; -} // namespace lsst::qserv::xrdsvc - // This header declarations -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { /// This class handles Http message from the czar to the worker. -class HttpWorkerCzarModule : public xrdsvc::HttpModule { +class HttpWorkerCzarModule : public wcomms::HttpModule { public: /// @note supported values for parameter 'subModuleName' are: /// 'QUERYJOB' - Convert an UberJob message into Tasks and a send channel. @@ -109,6 +105,6 @@ class HttpWorkerCzarModule : public xrdsvc::HttpModule { nlohmann::json _handleQueryStatus(std::string const& func); }; -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms -#endif // LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H +#endif // LSST_QSERV_WCOMMS_HTTPWORKERCZARMODULE_H diff --git a/src/wconfig/WorkerConfig.h b/src/wconfig/WorkerConfig.h index 6b0736bfff..a672b6fc12 100644 --- a/src/wconfig/WorkerConfig.h +++ b/src/wconfig/WorkerConfig.h @@ -162,9 +162,6 @@ class WorkerConfig { /// @return the name of a folder where query results will be stored std::string const resultsDirname() const { return _resultsDirname->getVal(); } - /// @return the port number of the worker XROOTD service for serving result files - uint16_t resultsXrootdPort() const { return _resultsXrootdPort->getVal(); } - /// The size int getQPoolSize() const { return _qPoolSize->getVal(); } @@ -334,8 +331,6 @@ class WorkerConfig { _configValMap, "sqlconnections", "reservedinteractivesqlconn", notReq, 50); CVTStrPtr _resultsDirname = util::ConfigValTStr::create(_configValMap, "results", "dirname", notReq, "/qserv/data/results"); - CVTUIntPtr _resultsXrootdPort = - util::ConfigValTUInt::create(_configValMap, "results", "xrootd_port", notReq, 1094); CVTUIntPtr _resultsNumHttpThreads = util::ConfigValTUInt::create(_configValMap, "results", "num_http_threads", notReq, 1); CVTBoolPtr _resultsCleanUpOnStart = diff --git a/src/wcontrol/Foreman.cc b/src/wcontrol/Foreman.cc index f990667359..8fd6fbb48b 100644 --- a/src/wcontrol/Foreman.cc +++ b/src/wcontrol/Foreman.cc @@ -171,7 +171,7 @@ Foreman::Foreman(wsched::BlendScheduler::Ptr const& scheduler, unsigned int pool Foreman::~Foreman() { LOGS(_log, LOG_LVL_DEBUG, "Foreman::~Foreman()"); - // It will take significant effort to have xrootd shutdown cleanly and this will never get called + // It will take significant effort to have qserv shutdown cleanly and this will never get called // until that happens. _pool->shutdownPool(); _httpServer->stop(); diff --git a/src/wcontrol/Foreman.h b/src/wcontrol/Foreman.h index dd1be58d32..19bb79aaec 100644 --- a/src/wcontrol/Foreman.h +++ b/src/wcontrol/Foreman.h @@ -188,6 +188,7 @@ class Foreman { /// A a counter of the XROOTD/SSI resources which are in use at any given moment /// of time by the worker. + /// TODO:UJ is this still tracking anything? Does removing it cause dashboard issues? std::shared_ptr const _resourceMonitor; /// BOOST ASIO services needed to run the HTTP server diff --git a/src/wdb/CMakeLists.txt b/src/wdb/CMakeLists.txt index 84a379a78a..7b4159ec35 100644 --- a/src/wdb/CMakeLists.txt +++ b/src/wdb/CMakeLists.txt @@ -24,7 +24,18 @@ FUNCTION(wdb_tests) add_executable(${TEST} ${TEST}.cc) target_link_libraries(${TEST} PUBLIC crypto - xrdsvc + global + http + mysql + protojson + sql + util + wbase + wconfig + wcontrol + wdb + wpublish + wsched Boost::unit_test_framework Threads::Threads ) diff --git a/src/wdb/ChunkResource.h b/src/wdb/ChunkResource.h index 68b83b0238..f60014ae70 100644 --- a/src/wdb/ChunkResource.h +++ b/src/wdb/ChunkResource.h @@ -50,9 +50,9 @@ #include "wdb/SQLBackend.h" // Forward declarations -namespace lsst::qserv { namespace wdb { +namespace lsst::qserv::wdb { class Task; -}} // namespace lsst::qserv::wdb +} // namespace lsst::qserv::wdb namespace lsst::qserv::wdb { diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index 3e2aa18d55..e5f0e6c92a 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -24,8 +24,7 @@ * @file * * @brief QueryRunner instances perform single-shot query execution with the - * result reflected in the db state or returned via a SendChannel. Works with - * new XrdSsi API. + * result reflected in the db state or returned via a SendChannel. * * @author Daniel L. Wang, SLAC; John Gates, SLAC */ diff --git a/src/wdb/QueryRunner.h b/src/wdb/QueryRunner.h index 0370cab501..803a26e9c3 100644 --- a/src/wdb/QueryRunner.h +++ b/src/wdb/QueryRunner.h @@ -27,8 +27,7 @@ * @file * * @brief QueryAction instances perform single-shot query execution with the - * result reflected in the db state or returned via a SendChannel. Works with - * new XrdSsi API. + * result reflected in the db state or returned via a SendChannel. * * @author Daniel L. Wang, SLAC */ diff --git a/src/wdb/SQLBackend.cc b/src/wdb/SQLBackend.cc index 53c0670132..ee5e8e3b97 100644 --- a/src/wdb/SQLBackend.cc +++ b/src/wdb/SQLBackend.cc @@ -241,8 +241,8 @@ void SQLBackend::_memLockRelease() { /// Exit the program immediately to reduce minimize possible problems. void SQLBackend::_exitDueToConflict(const std::string& msg) { - // This will likely not be clean exit, but clean exit is impossible - // with xrootd anyway. + // This will likely not be a clean exit. + // TODO:UJ try for a clean exit by calling WorkerMain::terminate(). _lockConflict = true; LOGS(_log, LOG_LVL_ERROR, msg); exit(EXIT_FAILURE); diff --git a/src/wmain/CMakeLists.txt b/src/wmain/CMakeLists.txt new file mode 100644 index 0000000000..1f6fac43e3 --- /dev/null +++ b/src/wmain/CMakeLists.txt @@ -0,0 +1,47 @@ +add_library(workerm SHARED) + +target_sources(workerm PRIVATE + WorkerMain.cc +) + + +target_link_libraries(workerm PUBLIC + boost_filesystem + boost_system + global + http + log + mysql + protojson + sql + wbase + wcomms + wconfig + wcontrol + wdb + wpublish + wsched + boost_program_options + ) + +install( + TARGETS workerm +) + +function(WORKERM_UTILS) + foreach(UTIL IN ITEMS ${ARGV}) + add_executable(${UTIL}) + target_sources(${UTIL} PRIVATE ${UTIL}.cc) + target_link_libraries(${UTIL} PRIVATE + global + workerm + ) + install(TARGETS ${UTIL}) + endforeach() +endfunction() + +workerm_utils( + qserv-worker-http +) + + diff --git a/src/xrdsvc/SsiService.cc b/src/wmain/WorkerMain.cc similarity index 86% rename from src/xrdsvc/SsiService.cc rename to src/wmain/WorkerMain.cc index 3f3335bb63..1a306df378 100644 --- a/src/xrdsvc/SsiService.cc +++ b/src/wmain/WorkerMain.cc @@ -22,7 +22,7 @@ */ // Class header -#include "xrdsvc/SsiService.h" +#include "wmain/WorkerMain.h" // System headers #include @@ -39,7 +39,6 @@ // Third-party headers #include -#include "XrdSsi/XrdSsiLogger.hh" // LSST headers #include "lsst/log/Log.h" @@ -55,44 +54,42 @@ #include "sql/SqlConnectionFactory.h" #include "util/common.h" #include "util/FileMonitor.h" -#include "util/HoldTrack.h" #include "wbase/Base.h" #include "wbase/FileChannelShared.h" #include "wconfig/WorkerConfig.h" #include "wconfig/WorkerConfigError.h" #include "wcontrol/Foreman.h" #include "wcontrol/SqlConnMgr.h" +#include "wcomms/HttpSvc.h" #include "wpublish/ChunkInventory.h" #include "wsched/BlendScheduler.h" #include "wsched/FifoScheduler.h" #include "wsched/GroupScheduler.h" #include "wsched/ScanScheduler.h" -#include "xrdsvc/HttpSvc.h" -#include "xrdsvc/XrdName.h" +#include "wcomms/HttpSvc.h" +#include "wcomms/XrdName.h" using namespace lsst::qserv; using namespace nlohmann; using namespace std; using namespace std::literals; -class XrdPosixCallBack; // Forward. - namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiService"); +LOG_LOGGER _log = LOG_GET("lsst.qserv.wmain.WorkerMain"); // add LWP to MDC in log messages void initMDC() { LOG_MDC("LWP", to_string(lsst::log::lwpID())); } int dummyInitMDC = LOG_MDC_INIT(initMDC); -std::shared_ptr makeChunkInventory(mysql::MySqlConfig const& mySqlConfig) { - xrdsvc::XrdName x; +std::shared_ptr makeChunkInventory(string const& workerName, + mysql::MySqlConfig const& mySqlConfig) { if (!mySqlConfig.dbName.empty()) { LOGS(_log, LOG_LVL_FATAL, "dbName must be empty to prevent accidental context"); throw runtime_error("dbName must be empty to prevent accidental context"); } auto conn = sql::SqlConnectionFactory::make(mySqlConfig); assert(conn); - auto inventory = make_shared(x.getName(), conn); + auto inventory = make_shared(workerName, conn); ostringstream os; os << "Paths exported: "; inventory->dbgPrint(os); @@ -123,7 +120,7 @@ void registryUpdateLoop(string const& id) { {"management-port", workerConfig->replicationHttpPort()}, {"management-host-name", util::get_current_host_fqdn()}}}}); string const requestContext = - "SsiService: '" + http::method2string(method) + "' request to '" + url + "'"; + "WorkerMain: '" + http::method2string(method) + "' request to '" + url + "'"; http::Client client(method, url, request.dump(), headers); while (true) { try { @@ -142,13 +139,29 @@ void registryUpdateLoop(string const& id) { } // namespace -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wmain { + +std::weak_ptr WorkerMain::_globalWorkerMain; +std::atomic WorkerMain::_setup{false}; -SsiService::SsiService(XrdSsiLogger* log) { - LOGS(_log, LOG_LVL_DEBUG, "SsiService starting..."); +WorkerMain::Ptr WorkerMain::setup() { + if (_setup.exchange(true)) { + throw util::Bug(ERR_LOC, "WorkerMain already setup when setup called again"); + } + auto ptr = Ptr(new WorkerMain()); + _globalWorkerMain = ptr; + return ptr; +} - util::HoldTrack::setup(10min); +std::shared_ptr WorkerMain::get() { + auto ptr = _globalWorkerMain.lock(); + if (ptr == nullptr) { + throw std::runtime_error("_globalWorkerMain is null"); + } + return ptr; +} +WorkerMain::WorkerMain() { auto const mySqlConfig = wconfig::WorkerConfig::instance()->getMySqlConfig(); if (not mysql::MySqlConnection::checkConnection(mySqlConfig)) { LOGS(_log, LOG_LVL_FATAL, "Unable to connect to MySQL using configuration:" << mySqlConfig); @@ -221,7 +234,7 @@ SsiService::SsiService(XrdSsiLogger* log) { string vectMinRunningSizesStr = workerConfig->getQPoolMinRunningSizes(); _foreman = wcontrol::Foreman::create(blendSched, poolSize, maxPoolThreads, mySqlConfig, queries, - ::makeChunkInventory(mySqlConfig), sqlConnMgr, qPoolSize, + ::makeChunkInventory(_name, mySqlConfig), sqlConnMgr, qPoolSize, maxPriority, vectRunSizesStr, vectMinRunningSizesStr); // Watch to see if the log configuration is changed. @@ -246,9 +259,8 @@ SsiService::SsiService(XrdSsiLogger* log) { // Start the control server for processing worker management requests sent // by the Replication System. Update the port number in the configuration // in case if the server is run on the dynamically allocated port. - _controlHttpSvc = HttpSvc::create(_foreman, workerConfig->replicationHttpPort(), - workerConfig->getCzarComNumHttpThreads()); - + _controlHttpSvc = wcomms::HttpSvc::create(_foreman, workerConfig->replicationHttpPort(), + workerConfig->getCzarComNumHttpThreads()); auto const port = _controlHttpSvc->start(); workerConfig->setReplicationHttpPort(port); @@ -258,13 +270,20 @@ SsiService::SsiService(XrdSsiLogger* log) { registryUpdateThread.detach(); } -SsiService::~SsiService() { - LOGS(_log, LOG_LVL_DEBUG, "SsiService dying."); - _controlHttpSvc->stop(); +void WorkerMain::waitForTerminate() { + unique_lock uniq(_terminateMtx); + _terminateCv.wait(uniq, [this]() { return _terminate; }); } -void SsiService::ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) { - LOGS(_log, LOG_LVL_ERROR, "SsiService::ProcessRequest got called"); +void WorkerMain::terminate() { + lock_guard lck(_terminateMtx); + _terminate = true; + _terminateCv.notify_all(); +} + +WorkerMain::~WorkerMain() { + LOGS(_log, LOG_LVL_INFO, "WorkerMain shutdown."); + _controlHttpSvc->stop(); } -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wmain diff --git a/src/xrdsvc/SsiService.h b/src/wmain/WorkerMain.h similarity index 53% rename from src/xrdsvc/SsiService.h rename to src/wmain/WorkerMain.h index f3ef7ab104..7461b0936a 100644 --- a/src/xrdsvc/SsiService.h +++ b/src/wmain/WorkerMain.h @@ -21,18 +21,16 @@ * see . */ -#ifndef LSST_QSERV_XRDSVC_SSISERVICE_H -#define LSST_QSERV_XRDSVC_SSISERVICE_H +#ifndef LSST_QSERV_WMAIN_WORKERMAIN_H +#define LSST_QSERV_WMAIN_WORKERMAIN_H // System headers +#include +#include #include +#include // Third-party headers -#include "XrdSsi/XrdSsiResource.hh" -#include "XrdSsi/XrdSsiService.hh" - -// Forward declarations -class XrdSsiLogger; namespace lsst::qserv::util { class FileMonitor; @@ -42,27 +40,41 @@ namespace lsst::qserv::wcontrol { class Foreman; } // namespace lsst::qserv::wcontrol -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wcomms { class HttpSvc; -} // namespace lsst::qserv::xrdsvc +} // namespace lsst::qserv::wcomms -namespace lsst::qserv::xrdsvc { +namespace lsst::qserv::wmain { -/// SsiService is an XrdSsiService implementation that implements a Qserv query -/// worker services -class SsiService : public XrdSsiService { +class WorkerMain { public: - /** Build a SsiService object - * @param log xrdssi logger - * @note take ownership of logger for now - */ - SsiService(XrdSsiLogger* log); - virtual ~SsiService(); + using Ptr = std::shared_ptr; + + /// Returns a pointer to the global instance. + /// @throw std::runtime_error if global pointer is null. + static std::shared_ptr get(); + static Ptr setup(); + + ~WorkerMain(); - /// Called by SSI framework to handle new requests - void ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) override; + std::string getName() const { return _name; } + + void terminate(); + void waitForTerminate(); private: + WorkerMain(); + + /// Weak pointer to allow global access without complicating lifetime issues. + static std::weak_ptr _globalWorkerMain; + + /// There should only be one WorkerMain, this prevents more than + /// one from being created. + static std::atomic _setup; + + /// Worker name, used in some database lookups. + std::string _name{"worker"}; + // The Foreman contains essential structures for adding and running tasks. std::shared_ptr _foreman; @@ -70,9 +82,13 @@ class SsiService : public XrdSsiService { std::shared_ptr _logFileMonitor; /// The HTTP server processing worker management requests. - std::shared_ptr _controlHttpSvc; -}; + std::shared_ptr _controlHttpSvc; -} // namespace lsst::qserv::xrdsvc + /// Set to true when the program should terminate. + bool _terminate = false; + std::mutex _terminateMtx; + std::condition_variable _terminateCv; +}; -#endif // LSST_QSERV_XRDSVC_SSISERVICE_H +} // namespace lsst::qserv::wmain +#endif // LSST_QSERV_WMAIN_WORKERMAIN_H diff --git a/src/wmain/qserv-worker-http.cc b/src/wmain/qserv-worker-http.cc new file mode 100644 index 0000000000..004382e1b2 --- /dev/null +++ b/src/wmain/qserv-worker-http.cc @@ -0,0 +1,82 @@ +/* + * LSST Data Management System + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +/** + * The CPP-HTTPLIB-based frontend for Czar. + */ + +// System headers +#include +#include +#include +#include +#include + +// Third party headers +#include "boost/program_options.hpp" + +// Qserv headers +#include "wmain/WorkerMain.cc" + +using namespace std; +namespace po = boost::program_options; +namespace qserv = lsst::qserv; + +namespace { +char const* const context = "[WORKER]"; +} // namespace + +int main(int argc, char* argv[]) { + po::options_description desc("", 120); + desc.add_options()("help,h", "Print this help message and exit."); + desc.add_options()("verbose,v", "Produce verbose output."); + desc.add_options()("config,c", po::value()->default_value("/config-etc/qserv-worker.cnf"), + "The configuration file."); + desc.add_options()("name,n", po::value()->default_value("worker"), "Worker name."); + + po::variables_map vm; + po::store(po::parse_command_line(argc, const_cast(argv), desc), vm); + po::notify(vm); + + string const configFilePath = vm["config"].as(); + + bool const verbose = vm.count("verbose") > 0; + if (verbose) { + cout << ::context << " Configuration file: " << configFilePath << "\n" << endl; + } + try { + cout << ::context << " Starting worker\n" + << " Configuration file: " << configFilePath << "\n" + << endl; + + auto const workerConfig = wconfig::WorkerConfig::create(configFilePath); + + // Lifetime of WorkerMain is controlled by wwMn. + auto wwMn = wmain::WorkerMain::setup(); + + wwMn->waitForTerminate(); + cout << ::context << " stopping worker" << endl; + } catch (exception const& ex) { + cerr << ::context << " The application failed, exception: " << ex.what() << endl; + return 1; + } + return 0; +} diff --git a/src/wpublish/CMakeLists.txt b/src/wpublish/CMakeLists.txt index d756bcad19..dafef6eafc 100644 --- a/src/wpublish/CMakeLists.txt +++ b/src/wpublish/CMakeLists.txt @@ -18,7 +18,18 @@ target_link_libraries(wpublish PUBLIC add_executable(testChunkInventory testChunkInventory.cc) target_link_libraries(testChunkInventory PUBLIC - xrdsvc + global + http + mysql + protojson + sql + util + wbase + wconfig + wcontrol + wdb + wpublish + wsched Boost::unit_test_framework Threads::Threads ) diff --git a/src/wpublish/ChunkInventory.cc b/src/wpublish/ChunkInventory.cc index 8bf735602d..085b114289 100644 --- a/src/wpublish/ChunkInventory.cc +++ b/src/wpublish/ChunkInventory.cc @@ -60,8 +60,6 @@ void fetchDbs(string const& instanceName, SqlConnection& sc, C& dbs) { shared_ptr resultP; // TODO we probably want a more elegant backoff mechanism than this. - // However, normally xrootd will fail & exit here if it can't connect so - // this is maybe just a little bit better than that. while (true) { LOGS(_log, LOG_LVL_DEBUG, "Launching query: " << query); resultP = sc.getQueryIter(query); diff --git a/src/wsched/CMakeLists.txt b/src/wsched/CMakeLists.txt index 0f4c2ea434..15ceaddf1f 100644 --- a/src/wsched/CMakeLists.txt +++ b/src/wsched/CMakeLists.txt @@ -23,8 +23,18 @@ install( add_executable(testSchedulers testSchedulers.cc) target_link_libraries(testSchedulers PUBLIC + global + http + mysql + protojson + sql + util + wbase + wconfig + wcontrol + wdb + wpublish wsched - xrdsvc Boost::unit_test_framework Threads::Threads ) diff --git a/src/xrdlog/CMakeLists.txt b/src/xrdlog/CMakeLists.txt deleted file mode 100644 index 011294e0c1..0000000000 --- a/src/xrdlog/CMakeLists.txt +++ /dev/null @@ -1,18 +0,0 @@ -add_library(xrdlog MODULE) - -target_sources(xrdlog PRIVATE - XrdLogging.cc -) - -target_include_directories(xrdlog PRIVATE - ${XROOTD_INCLUDE_DIRS} -) - -target_link_libraries(xrdlog PUBLIC - log - XrdSsiLib -) - -install( - TARGETS xrdlog DESTINATION ${CMAKE_INSTALL_LIBDIR} -) diff --git a/src/xrdlog/XrdLogging.cc b/src/xrdlog/XrdLogging.cc deleted file mode 100644 index 08fe1468b1..0000000000 --- a/src/xrdlog/XrdLogging.cc +++ /dev/null @@ -1,72 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2016 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// Implement logging hook to route xrootd/cmsd messages to our logger. -/// This source file has no header file. - -// Class header - -// System headers -#include - -// Third party headers -#include "XrdSsi/XrdSsiLogger.hh" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers - -/******************************************************************************/ -/* L o g g i n g I n t e r c e p t H o o k */ -/******************************************************************************/ - -namespace { - -const char* origin; - -void QservLogger(struct timeval const& mtime, unsigned long tID, const char* msg, int mlen) { - static log4cxx::spi::LocationInfo xrdLoc(origin, log4cxx::spi::LocationInfo::calcShortFileName(origin), - "", 0); - static LOG_LOGGER myLog = LOG_GET("lsst.qserv.xrdssi.msgs"); - - if (myLog.isInfoEnabled()) { - while (mlen && msg[mlen - 1] == '\n') --mlen; // strip all trailing newlines - std::string theMsg(msg, mlen); - lsst::log::Log::MDC("LWP", std::to_string(tID)); - myLog.logMsg(log4cxx::Level::getInfo(), xrdLoc, theMsg); - } -} - -XrdSsiLogger::MCB_t& ConfigLog() { - // Set the originator of the messages - origin = (getenv("XRDPROG") ? getenv("XRDPROG") : ""); - - // Configure the logging system - LOG_CONFIG(); - - // Return the address the logger to be used - return QservLogger; -} - -bool dummy = XrdSsiLogger::SetMCB(ConfigLog(), XrdSsiLogger::mcbServer); -} // namespace diff --git a/src/xrdsvc/CMakeLists.txt b/src/xrdsvc/CMakeLists.txt deleted file mode 100644 index fd90aee825..0000000000 --- a/src/xrdsvc/CMakeLists.txt +++ /dev/null @@ -1,37 +0,0 @@ -add_library(xrdsvc SHARED) - -target_sources(xrdsvc PRIVATE - HttpModule.cc - HttpMonitorModule.cc - HttpReplicaMgtModule.cc - HttpSvc.cc - HttpWorkerCzarModule.cc - SsiProvider.cc - SsiService.cc -) - -target_include_directories(xrdsvc PRIVATE - ${XROOTD_INCLUDE_DIRS} -) - -target_link_libraries(xrdsvc PUBLIC - boost_filesystem - boost_system - global - http - log - mysql - protojson - sql - wbase - wconfig - wcontrol - wdb - wpublish - wsched - XrdSsiLib -) - -install( - TARGETS xrdsvc -) diff --git a/src/xrdsvc/SsiProvider.cc b/src/xrdsvc/SsiProvider.cc deleted file mode 100644 index 53463b29e2..0000000000 --- a/src/xrdsvc/SsiProvider.cc +++ /dev/null @@ -1,162 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2016 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// Implement XrdSsiProviderServer to provide Qserv's SsiService -/// implementation. Link this file when building a plugin to be used as -/// ssi.svclib or oss.statlib. - -// Class header -#include "xrdsvc/SsiProvider.h" - -// System headers -#include -#include - -// Third party headers -#include "boost/filesystem.hpp" -#include "XrdSsi/XrdSsiCluster.hh" -#include "XrdSsi/XrdSsiLogger.hh" - -// LSST headers -#include "lsst/log/Log.h" - -// Qserv headers -#include "global/ResourceUnit.h" -#include "wconfig/WorkerConfig.h" -#include "wpublish/ChunkInventory.h" -#include "xrdsvc/XrdName.h" - -/******************************************************************************/ -/* G l o b a l s */ -/******************************************************************************/ - -// The following are global sysbols that point to an instance of our provider -// object. The SSI framework looks for these symbols when the shared library -// plug-in is loaded. The framework must find a valid provider object at load -// time or it will refuse to use the shared library. As the library is never -// unloaded, the object does not need to be deleted. -// -XrdSsiProvider* XrdSsiProviderServer = new lsst::qserv::xrdsvc::SsiProviderServer; - -XrdSsiProvider* XrdSsiProviderLookup = XrdSsiProviderServer; - -namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.xrdsvc.SsiProvider"); -} - -/******************************************************************************/ -/* D e s t r u c t o r */ -/******************************************************************************/ - -namespace lsst::qserv::xrdsvc { - -SsiProviderServer::~SsiProviderServer() {} - -/******************************************************************************/ -/* I n i t */ -/******************************************************************************/ - -bool SsiProviderServer::Init(XrdSsiLogger* logP, XrdSsiCluster* clsP, std::string cfgFn, std::string parms, - int argc, char** argv) { - lsst::qserv::xrdsvc::XrdName x; - - if (argc != 2) { - LOGS(_log, LOG_LVL_TRACE, "argc: " << argc); - LOGS(_log, LOG_LVL_FATAL, - "Incorrect xrdssi configuration, launch " - "xrootd with option '-+xrdssi /path/to/xrdssi/cfg/file'"); - exit(EXIT_FAILURE); - } - - LOGS(_log, LOG_LVL_DEBUG, "Qserv xrdssi plugin configuration file: " << argv[1]); - - std::string workerConfigFile = argv[1]; - auto const workerConfig = wconfig::WorkerConfig::create(workerConfigFile); - LOGS(_log, LOG_LVL_DEBUG, "Qserv xrdssi plugin configuration: " << *workerConfig); - - // Save the ssi logger as it places messages in another file than our log. - // - _logSsi = logP; - - // Save the cluster object as we will need to use it to inform the cluster - // when chunks come and go. We also can use it to schedule ourselves. The - // object or its absence will indicate whether or not we need to provide - // any service other than QueryResource(). - // - _cmsSsi = clsP; - - // We would process the configuration file (if present), any present - // parameters and the command line arguments. However, at the moment, we - // have nothing of interest in any of these arguments. So, we ignore them. - // - - // Herald our initialization - // - LOGS(_log, LOG_LVL_DEBUG, "SsiProvider initializing..."); - _logSsi->Msg("Qserv", "Provider Initializing"); - - // Initialize the inventory. We need to be able to handle QueryResource() - // calls either in the data provider and the metadata provider (we can be - // either one). - // - _chunkInventory.init(x.getName(), workerConfig->getMySqlConfig()); - - // If we are a data provider (i.e. xrootd) then we need to get the service - // object. It will print the exported paths. Otherwise, we need to print - // them here. This is kludgy and should be corrected when we transition to a - // single shared memory inventory object which should do this by itself. - // - if (clsP && clsP->DataContext()) { - _service.reset(new SsiService(logP)); - } else { - std::ostringstream ss; - ss << "Provider valid paths(ci): "; - _chunkInventory.dbgPrint(ss); - LOGS(_log, LOG_LVL_DEBUG, ss.str()); - _logSsi->Msg("Qserv", ss.str().c_str()); - } - - // We have completed full initialization. Return sucess. - // - return true; -} - -/******************************************************************************/ -/* Q u e r y R e s o u r c e */ -/******************************************************************************/ - -XrdSsiProvider::rStat SsiProviderServer::QueryResource(char const* rName, char const* contact) { - // Validate resource name based on its proposed type - return notPresent; -} - -void SsiProviderServer::ResourceAdded(const char* rName) { - // Handle resource based on its proposed type - return; -} - -void SsiProviderServer::ResourceRemoved(const char* rName) { - // Handle resource based on its proposed type - return; -} - -} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/SsiProvider.h b/src/xrdsvc/SsiProvider.h deleted file mode 100644 index 7969774796..0000000000 --- a/src/xrdsvc/SsiProvider.h +++ /dev/null @@ -1,79 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2015 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/// Implement XrdSsiProviderServer to provide Qserv's SsiService -/// implementation. Link this file when building a plugin to be used as -/// ssi.svclib. - -#ifndef LSST_QSERV_XRDSVC_SSIPROVIDER_H -#define LSST_QSERV_XRDSVC_SSIPROVIDER_H - -// System headers -#include -#include - -// Third-party headers -#include "XrdSsi/XrdSsiProvider.hh" - -// Qserv headers -#include "wpublish/ChunkInventory.h" -#include "xrdsvc/SsiService.h" - -// Forward declarations -class XrdSsiCluster; -class XrdSsiLogger; - -namespace lsst::qserv::xrdsvc { - -class SsiProviderServer : public XrdSsiProvider { -public: - XrdSsiService* GetService(XrdSsiErrInfo& eInfo, std::string const& contact, int oHold = 256) override { - return _service.get(); - } - - wpublish::ChunkInventory& GetChunkInventory() { return _chunkInventory; } - - XrdSsiCluster* GetClusterManager() { return _cmsSsi; } - - bool Init(XrdSsiLogger* logP, XrdSsiCluster* clsP, std::string cfgFn, std::string parms, int argc, - char** argv) override; - - rStat QueryResource(char const* rName, char const* contact = 0) override; - - void ResourceAdded(const char* rName) override; - - void ResourceRemoved(const char* rName) override; - - SsiProviderServer() : _cmsSsi(0), _logSsi(0) {} - virtual ~SsiProviderServer(); - -private: - wpublish::ChunkInventory _chunkInventory; - std::unique_ptr _service; - - XrdSsiCluster* _cmsSsi; - XrdSsiLogger* _logSsi; -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDSVC_SSIPROVIDER_H diff --git a/src/xrdsvc/XrdName.h b/src/xrdsvc/XrdName.h deleted file mode 100644 index 3bc61091f9..0000000000 --- a/src/xrdsvc/XrdName.h +++ /dev/null @@ -1,61 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2013 LSST Corporation. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -#ifndef LSST_QSERV_XRDFS_XRDNAME_H -#define LSST_QSERV_XRDFS_XRDNAME_H - -// System headers -#include -#include -#include - -namespace lsst::qserv::xrdsvc { - -/// XrdName is a small class that helps extract the name of a running xrootd (or -/// cmsd) instance. It does this by checking an environment variable that is -/// specified to be set during initialization of any xrootd/cmsd process. -class XrdName { -public: - XrdName() { - char const* name = std::getenv("XRDNAME"); - _setName(name ? name : "unknown"); - } - - std::string const& getName() const { return _name; } - -private: - void _setName(char const* name) { - _name.clear(); - // Discard non alpha-numeric characters other than '_' - for (char const* s = name; *s != '\0'; ++s) { - if (std::isalnum(*s) || *s == '_') { - _name.push_back(*s); - } - } - } - - std::string _name; -}; - -} // namespace lsst::qserv::xrdsvc - -#endif // LSST_QSERV_XRDFS_XRDNAME_H From 84107b25b543001f4e445cb992bffdd78331c1b3 Mon Sep 17 00:00:00 2001 From: Igor Gaponenko Date: Tue, 24 Jun 2025 23:52:27 -0700 Subject: [PATCH 13/15] Improved implementatin of the chunk map building algorithm --- src/replica/contr/ReplicationTask.cc | 65 ++++++++++++++++++++-------- src/replica/contr/ReplicationTask.h | 37 +++++++++++++--- 2 files changed, 77 insertions(+), 25 deletions(-) diff --git a/src/replica/contr/ReplicationTask.cc b/src/replica/contr/ReplicationTask.cc index a5d40bbfcd..e0e258906d 100644 --- a/src/replica/contr/ReplicationTask.cc +++ b/src/replica/contr/ReplicationTask.cc @@ -93,45 +93,71 @@ ReplicationTask::ReplicationTask(Controller::Ptr const& controller, _disableQservSync(disableQservSync), _forceQservSync(forceQservSync), _qservChunkMapUpdate(qservChunkMapUpdate), - _purge(purge) {} - -void ReplicationTask::_updateChunkMap() { - // Open MySQL connection using the RAII-style handler that would automatically - // abort the transaction should any problem occured when loading data into the table. - ConnectionHandler h; - try { - h.conn = Connection::open(Configuration::qservCzarDbParams("qservMeta")); - } catch (exception const& ex) { - error("failed to connect to the czar's database server, ex: " + string(ex.what())); - return; - } - QueryGenerator const g(h.conn); + _purge(purge), + _chunkMap(make_shared()) {} +bool ReplicationTask::_getChunkMap() { // Get info on known chunk replicas from the persistent store of the Replication system - // and package those into ready-to-ingest data. + // and package those into the new chunk disposition map. bool const allDatabases = true; string const emptyDatabaseFilter; bool const isPublished = true; bool const includeFileInfo = true; // need this to access tables sizes - vector rows; + shared_ptr newChunkMap = make_shared(); for (auto const& workerName : serviceProvider()->config()->workers()) { vector replicas; serviceProvider()->databaseServices()->findWorkerReplicas(replicas, workerName, emptyDatabaseFilter, allDatabases, isPublished, includeFileInfo); for (auto const& replica : replicas) { + // Incomplete replicas should not be used by Czar for query processing. + if (replica.status() != ReplicaInfo::Status::COMPLETE) continue; for (auto const& fileInfo : replica.fileInfo()) { if (fileInfo.isData() && !fileInfo.isOverlap()) { - rows.push_back(g.packVals(workerName, replica.database(), fileInfo.baseTable(), - replica.chunk(), fileInfo.size)); + (*newChunkMap)[workerName][replica.database()][fileInfo.baseTable()][replica.chunk()] = + fileInfo.size; } } } } - if (rows.empty()) { - warn("no replicas found in the persistent state of the Replication system"); + + // Update the current map if the new one is different from the current one. + if (*_chunkMap != *newChunkMap) { + _chunkMap = newChunkMap; + return true; + } + return false; +} + +void ReplicationTask::_updateChunkMap() { + if (!_getChunkMap() || _chunkMap->empty()) { + // No changes in the chunk map, or the map is still empty so there's + // nothing to do. return; } + // Open MySQL connection using the RAII-style handler that would automatically + // abort the transaction should any problem occured when loading data into the table. + ConnectionHandler h; + try { + h.conn = Connection::open(Configuration::qservCzarDbParams("qservMeta")); + } catch (exception const& ex) { + error("failed to connect to the czar's database server, ex: " + string(ex.what())); + return; + } + QueryGenerator const g(h.conn); + + // Pack the map into ready-to-ingest data. + vector rows; + for (auto const& [workerName, databases] : *_chunkMap) { + for (auto const& [databaseName, tables] : databases) { + for (auto const& [tableName, chunks] : tables) { + for (auto const [chunkId, size] : chunks) { + rows.push_back(g.packVals(workerName, databaseName, tableName, chunkId, size)); + } + } + } + } + // Get the limit for the length of the bulk insert queries. The limit is needed // to run the query generation. size_t maxQueryLength = 0; @@ -163,6 +189,7 @@ void ReplicationTask::_updateChunkMap() { error("failed to update chunk map in the Czar database, ex: " + string(ex.what())); return; } + info("chunk map has been updated in the Czar database"); } } // namespace lsst::qserv::replica diff --git a/src/replica/contr/ReplicationTask.h b/src/replica/contr/ReplicationTask.h index 2fb3c73bc4..9d272e8f8a 100644 --- a/src/replica/contr/ReplicationTask.h +++ b/src/replica/contr/ReplicationTask.h @@ -21,6 +21,11 @@ #ifndef LSST_QSERV_REPLICATIONTASK_H #define LSST_QSERV_REPLICATIONTASK_H +// System headers +#include +#include +#include + // Qserv headers #include "replica/contr/Task.h" @@ -56,7 +61,8 @@ class ReplicationTask : public Task { * up on the Qserv synchronization requests. * @param disableQservSync Disable replica synchronization at Qserv workers if 'true'. * @param forceQservSync Force chunk removal at worker resource collections if 'true'. - * @param qservChunkMapUpdate Update the chunk disposition map in Qserv's QMeta database if 'true'. + * @param qservChunkMapUpdate Enable updating the chunk disposition map in Qserv's QMeta database if + * 'true'. * @param replicationIntervalSec The number of seconds to wait in the end of each * iteration loop before to begin the new one. * @param purge Purge excess replicas if 'true'. @@ -77,17 +83,36 @@ class ReplicationTask : public Task { unsigned int qservSyncTimeoutSec, bool disableQservSync, bool forceQservSync, bool qservChunkMapUpdate, unsigned int replicationIntervalSec, bool purge); + /// Get info on known chunk replicas from the persistent store of the Replication system + /// and package those into the new chunk disposition map. Update the current map if the new one is + /// different from the current one. + /// @return 'true' if the map has been updated, 'false' otherwise. + bool _getChunkMap(); + + /// Update the chunk disposition map in QMeta when changes in the map are detected. void _updateChunkMap(); /// The maximum number of seconds to be waited before giving up /// on the Qserv synchronization requests. unsigned int const _qservSyncTimeoutSec; - bool const _disableQservSync; ///< Disable replica synchroization at Qserv workers if 'true'. - bool const _forceQservSync; ///< Force removal at worker resource collections if 'true'. - bool const - _qservChunkMapUpdate; /// Update the chunk disposition map in Qserv's QMeta database if 'true'. - bool const _purge; ///< Purge excess replicas if 'true'. + bool const _disableQservSync; ///< Disable replica synchroization at Qserv workers if 'true'. + bool const _forceQservSync; ///< Force removal at worker resource collections if 'true'. + bool const _qservChunkMapUpdate; ///< Enable updating the chunk disposition map in Qserv's QMeta + /// database if 'true'. + bool const _purge; ///< Purge excess replicas if 'true'. + + /// [worker] -> [database] -> [baseTable] -> [chunk] -> size + /// + /// The map represents the information on the replica disposition across Qserv workers. + /// The information is obtained from the persistent state of the Replication system on each + /// run of the task. The maps gets updated only if the new map is different from the current one. + /// + using ChunkMap = + std::map>>>; + + std::shared_ptr _chunkMap; ///< The current chunk disposition map }; } // namespace lsst::qserv::replica From fcd369802bf7ec8367bf464f90ba562b2472cb17 Mon Sep 17 00:00:00 2001 From: John Gates Date: Mon, 14 Jul 2025 11:48:06 -0700 Subject: [PATCH 14/15] Added memory/disk hybrid for transfering csv files. --- deploy/compose/docker-compose.yml | 8 + .../migrations/czar/migrate-11-to-12.sql | 31 ++- .../templates/proxy/etc/qserv-czar.cnf.jinja | 17 ++ src/cconfig/CzarConfig.h | 25 +- src/ccontrol/MergingHandler.cc | 84 ++---- src/ccontrol/MergingHandler.h | 19 +- src/ccontrol/UserQueryFactory.cc | 12 +- src/ccontrol/UserQueryFactory.h | 7 + src/ccontrol/UserQueryProcessList.cc | 4 +- src/ccontrol/UserQueryResultDelete.cc | 4 +- src/ccontrol/UserQuerySelect.cc | 25 +- src/ccontrol/UserQuerySelectCountStar.cc | 1 - src/ccontrol/UserQuerySelectCountStar.h | 1 - src/czar/ActiveWorker.cc | 3 +- src/czar/Czar.cc | 20 +- src/czar/Czar.h | 6 +- src/czar/CzarChunkMap.cc | 8 +- src/czar/CzarChunkMap.h | 2 +- src/czar/CzarRegistry.cc | 10 +- src/czar/HttpCzarSvc.cc | 10 + src/czar/HttpCzarSvc.h | 1 + src/czar/qserv-czar-http.cc | 36 ++- src/http/BaseModule.h | 2 +- src/mysql/CMakeLists.txt | 1 + src/mysql/CsvMemDisk.cc | 249 ++++++++++++++++++ src/mysql/CsvMemDisk.h | 214 +++++++++++++++ src/mysql/LocalInfile.cc | 15 -- src/protojson/testUberJobMsg.cc | 4 +- src/qdisp/Executive.cc | 66 ++++- src/qdisp/Executive.h | 54 ++-- src/qdisp/ResponseHandler.h | 5 - src/qdisp/UberJob.cc | 64 ++--- src/qdisp/UberJob.h | 19 +- src/qdisp/testQDisp.cc | 12 +- src/qmeta/QMetaMysql.cc | 1 - src/rproc/InfileMerger.cc | 57 ++-- src/rproc/InfileMerger.h | 36 +-- src/wcomms/HttpWorkerCzarModule.cc | 8 +- src/wmain/WorkerMain.cc | 99 +++---- src/wmain/WorkerMain.h | 6 +- 40 files changed, 869 insertions(+), 377 deletions(-) create mode 100644 src/mysql/CsvMemDisk.cc create mode 100644 src/mysql/CsvMemDisk.h diff --git a/deploy/compose/docker-compose.yml b/deploy/compose/docker-compose.yml index ff7245c4d8..ab0757b555 100644 --- a/deploy/compose/docker-compose.yml +++ b/deploy/compose/docker-compose.yml @@ -36,6 +36,7 @@ volumes: volume_czar_xrootd: volume_czar_home: volume_czar_cfg: + volume_czar_transfer: volume_czar_mariadb_data: volume_czar_mariadb_cfg: @@ -268,6 +269,10 @@ services: - type: volume source: volume_czar_mariadb_run target: /qserv/mariadb/run + - type: volume + source: volume_czar_transfer + target: /tmp + - << : *log-volume expose: - "3306" # for czar-mariadb @@ -304,6 +309,9 @@ services: - type: volume source: volume_czar_cfg target: /config-etc + - type: volume + source: volume_czar_transfer + target: /tmp - type: volume source: volume_czar_home target: /home/qserv diff --git a/python/lsst/qserv/schema/migrations/czar/migrate-11-to-12.sql b/python/lsst/qserv/schema/migrations/czar/migrate-11-to-12.sql index 0adc1075fc..2269c95413 100644 --- a/python/lsst/qserv/schema/migrations/czar/migrate-11-to-12.sql +++ b/python/lsst/qserv/schema/migrations/czar/migrate-11-to-12.sql @@ -1,14 +1,19 @@ --- ------------------------------------------------------------------- --- Rename table QStatsTmp into QProgress to reflect its purpose --- and add a foreign key constraint to QInfo table. --- This table tracks chunk processing progress of the running queries. --- ------------------------------------------------------------------- -ALTER TABLE QStatsTmp RENAME AS QProgress; -ALTER TABLE QProgress ADD CONSTRAINT `fk_queryId` FOREIGN KEY (`queryId`) REFERENCES `QInfo` (`queryId`) ON DELETE CASCADE ON UPDATE CASCADE; -ALTER TABLE QProgress COMMENT = 'Table to track chunk processing progress of the running queries.'; +-- ----------------------------------------------------- +-- Table `chunkMap` +-- ----------------------------------------------------- +CREATE TABLE IF NOT EXISTS `chunkMap` ( + `worker` VARCHAR(256) NOT NULL COMMENT 'A unique identifier of a worker hosting the chunk replica', + `database` VARCHAR(256) NOT NULL COMMENT 'The name of a database', + `table` VARCHAR(256) NOT NULL COMMENT 'The name of a table', + `chunk` INT UNSIGNED NOT NULL COMMENT 'The number of a chunk', + `size` BIGINT UNSIGNED NOT NULL COMMENT 'The size of a chunk') +ENGINE = InnoDB +COMMENT = 'Chunk disposition across workers'; --- ------------------------------------------------------------------- --- Drop the QWorker table as it is no longer needed. --- This table was used to track worker nodes and their statuses. --- ------------------------------------------------------------------- -DROP TABLE IF EXISTS QWorker; +-- ----------------------------------------------------- +-- Table `chunkMapStatus` +-- ----------------------------------------------------- +CREATE TABLE IF NOT EXISTS `chunkMapStatus` ( + `update_time` TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'The most recent update time of the map') +ENGINE = InnoDB +COMMENT = 'Satus info on the chunk map'; \ No newline at end of file diff --git a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja index 1856bcc6a3..35f4cd8128 100644 --- a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja +++ b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja @@ -25,12 +25,29 @@ port = {{ czar_db_port }} # Any table in resultdb that hasn't been updated in this many days is deleted. oldestResultKeptDays = 7 +# Either this should be changed to a high performance docker volume directory +# or /tmp should be mounted as a high performance docker volume directory +# to avoid using limited docker memory to store the contents. +transferDir = /tmp + # maximum number of connection retries to SQL databse (per connection attempt) maxsqlconnectionattempts = 10 # maximum user query result size in MB maxtablesize_mb = 5100 +# maximum number of MB of concurrent csv transfer files allowed to be kept in +# memory, after this point the will be temporarily written to disk. +# 0 is used for testing. 10000 is usually reasonable. +maxTransferMemMB = 0 + +# minimum number of MB for each csv transfer file to be kept in memory +# before possibly going to disk. +# 0 for testing, up to 10 should be reasonable. +transferMinMBInMem = 0 + + + # database connection for QMeta database [qmeta] diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 89964b0bbe..a07fe779a5 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -130,8 +130,16 @@ class CzarConfig { /// Getters for result aggregation options. int getMaxTableSizeMB() const { return _maxTableSizeMB->getVal(); } int getMaxSqlConnectionAttempts() const { return _maxSqlConnectionAttempts->getVal(); } + unsigned int getMaxTransferMemMB() const { return _resultMaxTransferMemMB->getVal(); } + /// Return the transfer directory. This is customizable to allow for a + /// high performance volume. + std::string getTransferDir() const { return _resultTransferDir->getVal(); } - /// The size of the TCP connection pool witin the client API that is used + /// Return the minimum amount of memory per UberJob to keep in memory. This much transfer + /// data will be stored in memory regardless of other conditions. + unsigned int getTransferMinMBInMem() const { return _resultTransferMinMBInMem->getVal(); } + + /// The size of the TCP connection pool within the client API that is used /// by the merger to pool result files from workers via the HTTP protocol. int getResultMaxHttpConnections() const { return _resultMaxHttpConnections->getVal(); } @@ -169,13 +177,6 @@ class CzarConfig { /// the method then the monitoring will be disabled. unsigned int czarStatsUpdateIvalSec() const { return _czarStatsUpdateIvalSec->getVal(); } - /// @return The maximum retain period for keeping in memory the relevant metrics - /// captured by the Czar monitoring system. If 0 is returned by the method then - /// query history archiving will be disabled. - /// @note Setting the limit too high may be potentially result in runing onto - /// the OOM situation. - unsigned int czarStatsRetainPeriodSec() const { return _czarStatsRetainPeriodSec->getVal(); } - /// A worker is considered fully ALIVE if the last update from the worker has been /// heard in less than _activeWorkerTimeoutAliveSecs seconds. int getActiveWorkerTimeoutAliveSecs() const { return _activeWorkerTimeoutAliveSecs->getVal(); } @@ -306,6 +307,14 @@ class CzarConfig { CVTIntPtr _oldestAsyncResultKeptSeconds = util::ConfigValTInt::create( _configValMap, "resultdb", "oldestAsyncResultKeptSeconds", notReq, 3600); + // This must be larger than _maxTableSizeMB when using the "memory" TransferMethod + CVTUIntPtr _resultMaxTransferMemMB = + util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); + CVTStrPtr _resultTransferDir = + util::ConfigValTStr::create(_configValMap, "resultdb", "transferDir", notReq, "/tmp"); + CVTUIntPtr _resultTransferMinMBInMem = + util::ConfigValTUInt::create(_configValMap, "resultdb", "transferMinMBInMem", notReq, 10); + /// Get all the elements in the css section. CVTStrPtr _cssTechnology = util::ConfigValTStr::create(_configValMap, "css", "technology", notReq, "mysql"); diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index e370929427..3385e2aace 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -39,17 +39,17 @@ #include "lsst/log/Log.h" // Qserv headers +#include "cconfig/CzarConfig.h" #include "ccontrol/msgCode.h" #include "global/clock_defs.h" #include "global/debugUtil.h" #include "http/Client.h" #include "http/ClientConnPool.h" #include "http/Method.h" -#include "mysql/CsvBuffer.h" +#include "mysql/CsvMemDisk.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" -#include "qdisp/QueryRequest.h" #include "qdisp/UberJob.h" #include "rproc/InfileMerger.h" #include "util/Bug.h" @@ -84,7 +84,6 @@ lsst::qserv::TimeCountTracker::CALLBACKFUNC const reportFileRecvRate = } }; - string readHttpFileAndMerge(lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, size_t fileSize, function const& messageIsReady, shared_ptr const& httpConnPool) { @@ -180,51 +179,6 @@ MergingHandler::MergingHandler(std::shared_ptr const& merge MergingHandler::~MergingHandler() { LOGS(_log, LOG_LVL_TRACE, __func__); } - -bool MergingHandler::flush(proto::ResponseSummary const& resp) { - _wName = resp.wname(); - - // This is needed to ensure the job query would be staying alive for the duration - // of the operation to prevent inconsistency within the application. - auto const jobQuery = getJobQuery().lock(); - if (jobQuery == nullptr) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobQuery was NULL"); - return false; - } - auto const jobQuery = std::dynamic_pointer_cast(jobBase); - - LOGS(_log, LOG_LVL_TRACE, - "MergingHandler::" << __func__ << " jobid=" << resp.jobid() << " transmitsize=" - << resp.transmitsize() << " rowcount=" << resp.rowcount() << " rowSize=" - << " attemptcount=" << resp.attemptcount() << " errorcode=" << resp.errorcode() - << " errormsg=" << resp.errormsg()); - - if (resp.errorcode() != 0 || !resp.errormsg().empty()) { - _error = util::Error(resp.errorcode(), resp.errormsg(), util::ErrorCode::MYSQLEXEC); - _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); - LOGS(_log, LOG_LVL_ERROR, - "MergingHandler::" << __func__ << " error from worker:" << resp.wname() << " error: " << _error); - // This way we can track if the worker has reported this error. The current implementation - // requires the large result size to be reported as an error via the InfileMerger regardless - // of an origin of the error (Czar or the worker). Note that large results can be produced - // by the Czar itself, e.g., when the aggregate result of multiple worker queries is too large - // or by the worker when the result set of a single query is too large. - // The error will be reported to the Czar as a part of the response summary. - if (resp.errorcode() == util::ErrorCode::WORKER_RESULT_TOO_LARGE) { - _infileMerger->setResultSizeLimitExceeded(); - } - return false; - } - - bool const success = _merge(resp, jobQuery); - if (success) { - _infileMerger->mergeCompleteFor(resp.jobid()); - qdisp::CzarStats::get()->addTotalRowsRecv(resp.rowcount()); - qdisp::CzarStats::get()->addTotalBytesRecv(resp.transmitsize()); - } - return success; -} - void MergingHandler::errorFlush(std::string const& msg, int code) { _setError(code, msg, util::ErrorCode::RESULT_IMPORT); // Might want more info from result service. @@ -243,13 +197,8 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } if (fileSize == 0) return qdisp::MergeEndStatus(true); - - // Read from the http stream and push records into the CSV stream in a separate thread. - // Note the fixed capacity of the stream which allows up to 2 records to be buffered - // in the stream. This is enough to hide the latency of the HTTP connection and - // the time needed to read the file. - auto csvStream = mysql::CsvStream::create(2); - _csvStream = csvStream; + auto csvMemDisk = mysql::CsvMemDisk::create(fileSize, uberJob->getQueryId(), uberJob->getUjId()); + _csvMemDisk = csvMemDisk; // This must be after setting _csvStream to avoid cancelFileMerge() // race issues, and it needs to be before the thread starts. @@ -259,21 +208,21 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } string fileReadErrorMsg; - thread csvThread([uberJob, csvStream, fileUrl, fileSize, &fileReadErrorMsg]() { + auto transferFunc = [&]() { size_t bytesRead = 0; fileReadErrorMsg = ::readHttpFileAndMerge( uberJob, fileUrl, fileSize, - [uberJob, csvStream, fileSize, &bytesRead](char const* buf, uint32_t size) { + [&](char const* buf, uint32_t size) { bool last = false; if (buf == nullptr || size == 0) { last = true; } else { - csvStream->push(buf, size); + csvMemDisk->push(buf, size); bytesRead += size; last = bytesRead >= fileSize; } if (last) { - csvStream->push(nullptr, 0); + csvMemDisk->push(nullptr, 0); } }, MergingHandler::_getHttpConnPool()); @@ -281,24 +230,24 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // It may be needed to unblock the table merger which may be still attempting to read // from the CSV stream. if (!fileReadErrorMsg.empty()) { - csvStream->push(nullptr, 0); + csvMemDisk->push(nullptr, 0); } - }); + }; + csvMemDisk->transferDataFromWorker(transferFunc); // Attempt the actual merge. - bool fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvStream); + bool fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvMemDisk); if (!fileMergeSuccess) { LOGS(_log, LOG_LVL_WARN, __func__ << " merge failed"); util::Error const& err = _infileMerger->getError(); _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg(), util::ErrorCode::RESULT_IMPORT); } - if (csvStream->getContaminated()) { + if (csvMemDisk->getContaminated()) { LOGS(_log, LOG_LVL_ERROR, __func__ << " merge stream contaminated"); fileMergeSuccess = false; _setError(ccontrol::MSG_RESULT_ERROR, "merge stream contaminated", util::ErrorCode::RESULT_IMPORT); } - csvThread.join(); if (!fileReadErrorMsg.empty()) { LOGS(_log, LOG_LVL_WARN, __func__ << " result file read failed"); _setError(ccontrol::MSG_HTTP_RESULT, fileReadErrorMsg, util::ErrorCode::RESULT_IMPORT); @@ -309,14 +258,14 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber if (!mergeEStatus.success) { // This error check needs to come after the csvThread.join() to ensure writing // is finished. If any bytes were written, the result table is ruined. - mergeEStatus.contaminated = csvStream->getBytesWritten() > 0; + mergeEStatus.contaminated = csvMemDisk->getBytesFetched() > 0; } return mergeEStatus; } void MergingHandler::cancelFileMerge() { - auto csvStrm = _csvStream.lock(); + auto csvStrm = _csvMemDisk.lock(); if (csvStrm != nullptr) { csvStrm->cancel(); } @@ -342,9 +291,6 @@ qdisp::MergeEndStatus MergingHandler::flushHttp(string const& fileUrl, uint64_t "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); qdisp::MergeEndStatus mergeStatus = _mergeHttp(uberJob, fileUrl, fileSize); - if (mergeStatus.success) { - _infileMerger->mergeCompleteFor(uberJob->getUjId()); - } return mergeStatus; } diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 68bdb2f632..f01e9726a8 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -37,7 +37,7 @@ class ClientConnPool; } // namespace lsst::qserv::http namespace lsst::qserv::mysql { -class CsvStream; +class CsvMemDisk; } // namespace lsst::qserv::mysql namespace lsst::qserv::qdisp { @@ -91,20 +91,21 @@ class MergingHandler : public qdisp::ResponseHandler { /// Set error code and string. void _setError(int code, std::string const& msg, int errorState); - /// Check if the query is no longer active. - /// This is used to prevent the query from being processed after it has been cancelled - /// or finished for any reason. - /// @param jobQuery the query to check - /// @return true if the query is no longer active - bool _queryIsNoLongerActive(std::shared_ptr const& jobQuery) const; + // All instances of the HTTP client class are members of the same pool. This allows + // connection reuse and a significant reduction of the kernel memory pressure. + // Note that the pool gets instantiated at the very first call to method _getHttpConnPool() + // because the instantiation depends on the availability of the Czar configuration. + static std::shared_ptr const& _getHttpConnPool(); + static std::shared_ptr _httpConnPool; + static std::mutex _httpConnPoolMutex; std::shared_ptr _infileMerger; ///< Merging delegate std::atomic _errorSet{false}; ///< Set to true when an error is set. bool _flushed{false}; ///< flushed to InfileMerger? std::string _wName{"~"}; ///< worker name - std::weak_ptr _executive; ///< Weak pointer to the executive for errors. - std::weak_ptr _csvStream; ///< Weak pointer to cancel infile merge. + std::weak_ptr _executive; ///< Weak pointer to the executive for errors. + std::weak_ptr _csvMemDisk; ///< Weak pointer to cancel infile merge. }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 61ac3f10f4..b38072e33b 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -135,7 +135,7 @@ std::shared_ptr _makeUserQueryProcessList(query::SelectStmt::Ptr& stm LOGS(_log, LOG_LVL_DEBUG, "SELECT query is a PROCESSLIST"); try { return std::make_shared(stmt, sharedResources->qMetaSelect, - sharedResources->qMetaCzarId, userQueryId, resultDb); + sharedResources->czarId, userQueryId, resultDb); } catch (std::exception const& exc) { return std::make_shared(exc.what()); } @@ -297,6 +297,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st } auto stmt = parser->getSelectStmt(); + std::lock_guard focatoryLock(_factoryMtx); // handle special database/table names if (_stmtRefersToProcessListTable(stmt, defaultDb)) { return _makeUserQueryProcessList(stmt, _userQuerySharedResources, userQueryId, resultDb, aQuery, @@ -331,7 +332,6 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st } // This is a regular SELECT for qserv - // Currently using the database for results to get schema information. auto qs = std::make_shared(_userQuerySharedResources->css, _userQuerySharedResources->databaseModels, defaultDb, @@ -372,7 +372,8 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st qs, messageStore, executive, _userQuerySharedResources->databaseModels, infileMergerConfig, _userQuerySharedResources->secondaryIndex, _userQuerySharedResources->queryMetadata, _userQuerySharedResources->queryProgress, _userQuerySharedResources->czarId, errorExtra, - async, resultDb); + async, resultDb, uberJobMaxChunks); + if (sessionValid) { uq->qMetaRegister(resultLocation, msgTableName); uq->setupMerger(); @@ -381,11 +382,13 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st } return uq; } else if (UserQueryType::isSelectResult(query, userJobId)) { + std::lock_guard factoryLock(_factoryMtx); auto uq = std::make_shared(userJobId, _userQuerySharedResources->czarId, _userQuerySharedResources->queryMetadata); LOGS(_log, LOG_LVL_DEBUG, "make UserQueryAsyncResult: userJobId=" << userJobId); return uq; } else if (UserQueryType::isShowProcessList(query, full)) { + std::lock_guard factoryLock(_factoryMtx); LOGS(_log, LOG_LVL_DEBUG, "make UserQueryProcessList: full=" << (full ? 'y' : 'n')); try { return std::make_shared(full, _userQuerySharedResources->qMetaSelect, @@ -395,6 +398,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st return std::make_shared(exc.what()); } } else if (UserQueryType::isCall(query)) { + std::lock_guard factoryLock(_factoryMtx); auto parser = std::make_shared( query, _userQuerySharedResources->makeUserQueryResources(userQueryId, resultDb)); return parser->getUserQuery(); @@ -406,6 +410,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st return std::make_shared(std::string("ParseException:") + e.what()); } auto uq = parser->getUserQuery(); + std::lock_guard factoryLock(_factoryMtx); auto setQuery = std::static_pointer_cast(uq); if (setQuery->varName() == "QSERV_ROW_COUNTER_OPTIMIZATION") { _useQservRowCounterOptimization = setQuery->varValue() != "0"; @@ -417,6 +422,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st } return uq; } else { + std::lock_guard factoryLock(_factoryMtx); // something that we don't recognize auto uq = std::make_shared("Invalid or unsupported query: " + query); return uq; diff --git a/src/ccontrol/UserQueryFactory.h b/src/ccontrol/UserQueryFactory.h index 78199c60c7..35cecbdce7 100644 --- a/src/ccontrol/UserQueryFactory.h +++ b/src/ccontrol/UserQueryFactory.h @@ -103,6 +103,13 @@ class UserQueryFactory : private boost::noncopyable { boost::asio::io_service _asioIoService; std::unique_ptr _asioWork; std::unique_ptr _asioTimerThread; + + /// This protects the CSS calls inside qs->analyzeQuery(query, stmt); as well + /// as some changes UserQueries may be making to databases. + /// TODO: It would be safer to have CSS be thread safe. + /// TODO: Go through all of the affected database interactions and make sure + /// they are thread safe without this mutex. + std::mutex _factoryMtx; }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQueryProcessList.cc b/src/ccontrol/UserQueryProcessList.cc index 83ff05eb7a..070d6b9d55 100644 --- a/src/ccontrol/UserQueryProcessList.cc +++ b/src/ccontrol/UserQueryProcessList.cc @@ -67,7 +67,7 @@ UserQueryProcessList::UserQueryProcessList(std::shared_ptr co qmeta::CzarId czarId, std::string const& userQueryId, std::string const& resultDb) : _qMetaSelect(qMetaSelect), - _qMetaCzarId(qMetaCzarId), + _czarId(czarId), _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), _resultDb(resultDb) { @@ -94,7 +94,7 @@ UserQueryProcessList::UserQueryProcessList(bool full, std::shared_ptr()), _resultTableName(::g_nextResultTableId(userQueryId)), _resultDb(resultDb) { diff --git a/src/ccontrol/UserQueryResultDelete.cc b/src/ccontrol/UserQueryResultDelete.cc index 51024329b7..f798b91b45 100644 --- a/src/ccontrol/UserQueryResultDelete.cc +++ b/src/ccontrol/UserQueryResultDelete.cc @@ -39,7 +39,7 @@ #include "cconfig/CzarConfig.h" #include "qmeta/Exceptions.h" #include "qmeta/QMeta.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" #include "sql/SqlErrorObject.h" @@ -57,7 +57,7 @@ namespace lsst::qserv::ccontrol { UserQueryResultDelete::UserQueryResultDelete(shared_ptr const& queryResources, string const& value) - : _value(value), _queryResources(queryResources), _messageStore(make_shared()) {} + : _value(value), _queryResources(queryResources), _messageStore(make_shared()) {} void UserQueryResultDelete::submit() { LOGS(_log, LOG_LVL_DEBUG, "UserQueryResultDelete::submit: " << _value); diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 8238013221..3887001eab 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -241,7 +241,7 @@ void UserQuerySelect::submit() { LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect beginning submission"); assert(_infileMerger); - _ttn = std::make_shared(_qMetaQueryId, _qSession->getOriginal()); + _ttn = std::make_shared(_queryId, _qSession->getOriginal()); std::vector chunks; std::mutex chunksMtx; JobId sequence = 0; @@ -269,10 +269,7 @@ void UserQuerySelect::submit() { ++i) { auto& chunkSpec = *i; - std::function funcBuildJob = [this, sequence, // sequence must be a copy - &chunkSpec, &queryTemplates, &ttn, - &taskMsgFactory](util::CmdData*) { - QSERV_LOGCONTEXT_QUERY(_queryId); + QSERV_LOGCONTEXT_QUERY(_queryId); // TODO:UJ The template(s) is generated here and later it is compared to other // templates. It would be better to create the list of query templates here @@ -299,7 +296,7 @@ void UserQuerySelect::submit() { ResourceUnit ru; ru.setAsDbChunk(cs->db, cs->chunkId); qdisp::JobDescription::Ptr jobDesc = - qdisp::JobDescription::create(_qMetaCzarId, exec->getId(), sequence, ru, cs); + qdisp::JobDescription::create(_czarId, exec->getId(), sequence, ru, cs); auto job = exec->add(jobDesc); ++sequence; } @@ -312,17 +309,11 @@ void UserQuerySelect::submit() { LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); // TODO:UJ Waiting for all jobs to start may not be needed anymore? exec->waitForAllJobsToStart(); - - // we only care about per-chunk info for ASYNC queries - if (_async) { - std::lock_guard lock(chunksMtx); - _qMetaAddChunks(chunks); - } } void UserQuerySelect::buildAndSendUberJobs() { // TODO:UJ Is special handling needed for the dummy chunk, 1234567890 ? - string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); + string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_queryId)); LOGS(_log, LOG_LVL_DEBUG, funcN << " start " << _uberJobMaxChunks); // Ensure `_monitor()` doesn't do anything until everything is ready. @@ -473,8 +464,7 @@ void UserQuerySelect::buildAndSendUberJobs() { string uberResultName = _ttn->make(ujId); auto respHandler = ccontrol::MergingHandler::Ptr(new ccontrol::MergingHandler(_infileMerger, exec)); - auto uJob = qdisp::UberJob::create(exec, respHandler, exec->getId(), ujId, _qMetaCzarId, - targetWorker); + auto uJob = qdisp::UberJob::create(exec, respHandler, exec->getId(), ujId, _czarId, targetWorker); uJob->setWorkerContactInfo(wInfUJ->wInf); wInfUJ->uberJobPtr = uJob; }; @@ -534,7 +524,7 @@ QueryState UserQuerySelect::join() { // Since all data are in, run final SQL commands like GROUP BY. size_t collectedBytes = 0; int64_t finalRows = 0; - bool const resultSizeLimitExceeded = _infileMerger->resultSizeLimitExceeded(); + bool const resultSizeLimitExceeded = exec->resultSizeLimitExceeded(); if (!_infileMerger->finalize(collectedBytes, finalRows)) { successful = false; LOGS(_log, LOG_LVL_ERROR, "InfileMerger::finalize failed"); @@ -574,7 +564,6 @@ QueryState UserQuerySelect::join() { auto const status = resultSizeLimitExceeded ? qmeta::QInfo::FAILED_LR : qmeta::QInfo::FAILED; _qMetaUpdateStatus(status, collectedRows, collectedBytes, finalRows); LOGS(_log, LOG_LVL_ERROR, "Joined everything (failure!) QID=" << getQueryId()); - operation = proto::QueryManagement::CANCEL; state = ERROR; } auto const czarConfig = cconfig::CzarConfig::instance(); @@ -807,7 +796,7 @@ void UserQuerySelect::qMetaRegister(std::string const& resultLocation, std::stri auto exec = _executive; if (exec != nullptr) { - exec->setQueryId(_qMetaQueryId); + exec->setQueryId(_queryId); } else { LOGS(_log, LOG_LVL_WARN, "No Executive, assuming invalid query"); } diff --git a/src/ccontrol/UserQuerySelectCountStar.cc b/src/ccontrol/UserQuerySelectCountStar.cc index 0396d34e7e..47767146c1 100644 --- a/src/ccontrol/UserQuerySelectCountStar.cc +++ b/src/ccontrol/UserQuerySelectCountStar.cc @@ -62,7 +62,6 @@ UserQuerySelectCountStar::UserQuerySelectCountStar(std::string query, : _qMetaSelect(qMetaSelect), _queryMetadata(queryMetadata), _messageStore(std::make_shared()), - _resultTableName(::g_nextResultTableId(userQueryId)), _userQueryId(userQueryId), _rowsTable(rowsTable), _resultDb(resultDb), diff --git a/src/ccontrol/UserQuerySelectCountStar.h b/src/ccontrol/UserQuerySelectCountStar.h index f249bf3db8..f2c3246755 100644 --- a/src/ccontrol/UserQuerySelectCountStar.h +++ b/src/ccontrol/UserQuerySelectCountStar.h @@ -119,7 +119,6 @@ class UserQuerySelectCountStar : public UserQuery { std::shared_ptr _messageStore; std::string _resultTable; std::string _resultLoc; ///< Result location - std::string _resultTableName; std::string _userQueryId; std::string _rowsTable; std::string _resultDb; diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index 81d011532c..12aae1fce6 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -288,7 +288,8 @@ void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap auto iter = _awMap.find(wcKey); if (iter == _awMap.end()) { auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ActiveWorker created for " << wcKey); + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " ActiveWorker created for " << wcKey << " " << newAW->dump()); _awMap[wcKey] = newAW; if (_czarCancelAfterRestart) { newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 4af93623a2..b7cf2ce04d 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -53,6 +53,7 @@ #include "http/ClientConnPool.h" #include "http/MetaModule.h" #include "http/Method.h" +#include "mysql/CsvMemDisk.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qproc/DatabaseModels.h" @@ -160,7 +161,8 @@ Czar::Czar(string const& configFilePath, string const& czarName) _uqFactory(), _clientToQuery(), _monitorSleepTime(_czarConfig->getMonitorSleepTimeMilliSec()), - _activeWorkerMap(new ActiveWorkerMap(_czarConfig)) { + _activeWorkerMap(new ActiveWorkerMap(_czarConfig)), + _fqdn(util::getCurrentHostFqdnBlocking()) { // set id counter to milliseconds since the epoch, mod 1 year. struct timeval tv; gettimeofday(&tv, nullptr); @@ -177,6 +179,15 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->czarId); + CzarIdType czarId = _czarConfig->id(); + size_t const MB_SIZE_BYTES = 1024 * 1024; + size_t maxResultTableSizeBytes = _czarConfig->getMaxTableSizeMB() * MB_SIZE_BYTES; + size_t maxMemToUse = _czarConfig->getMaxTransferMemMB() * MB_SIZE_BYTES; + string const transferDirectory = _czarConfig->getTransferDir(); + std::size_t const transferMinBytesInMem = _czarConfig->getTransferMinMBInMem() * MB_SIZE_BYTES; + mysql::TransferTracker::setup(maxMemToUse, transferDirectory, transferMinBytesInMem, + maxResultTableSizeBytes, czarId); + // Tell workers to cancel any queries that were submitted before this restart of Czar. // Figure out which query (if any) was recorded in Czar databases before the restart. // The id will be used as the high-watermark for queries that need to be cancelled. @@ -300,11 +311,8 @@ SubmitResult Czar::submitQuery(string const& query, map const& h // make new UserQuery // this is atomic - ccontrol::UserQuery::Ptr uq; - { - lock_guard lock(_mutex); - uq = _uqFactory->newUserQuery(query, defaultDb, getQdispPool(), userQueryId, msgTableName, resultDb); - } + ccontrol::UserQuery::Ptr uq = + _uqFactory->newUserQuery(query, defaultDb, getQdispPool(), userQueryId, msgTableName, resultDb); // Add logging context with query ID QSERV_LOGCONTEXT_QUERY(uq->getQueryId()); diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 67a5261a34..c250daab15 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -123,7 +123,6 @@ class Czar { */ static Ptr getCzar() { return _czar; } - /// Remove all old tables in the qservResult database. void removeOldResultTables(); @@ -161,6 +160,8 @@ class Czar { std::shared_ptr getCommandHttpPool() const { return _commandHttpPool; } + std::string const& getFqdn() const { return _fqdn; } + /// Startup time of czar, sent to workers so they can detect that the czar was /// was restarted when this value changes. static uint64_t const czarStartupTime; @@ -254,6 +255,9 @@ class Czar { /// Pool of http client connections for sending commands (UberJobs /// and worker status requests). std::shared_ptr _commandHttpPool; + + /// FQDN for this czar. + std::string const _fqdn; }; } // namespace lsst::qserv::czar diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 3f84b3b928..3cf204b516 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -139,10 +139,10 @@ void CzarChunkMap::verify(string const& familyName) const { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " family=" << familyName << " verified"); } -string CzarChunkMap::dumpChunkMap(ChunkMap const& chunkMap) { +string CzarChunkMap::dumpChunkMap() const { stringstream os; os << "ChunkMap{"; - for (auto const& [cId, cDataPtr] : chunkMap) { + for (auto const& [cId, cDataPtr] : *_chunkMap) { os << "(cId=" << cId << ":"; os << ((cDataPtr == nullptr) ? "null" : cDataPtr->dump()) << ")"; } @@ -355,6 +355,10 @@ bool CzarFamilyMap::_read() { verify(familyMapPtr); + for (auto const& [fam, ccMap] : *familyMapPtr) { + LOGS(_log, LOG_LVL_INFO, "{family=" << fam << "{" << ccMap->dumpChunkMap() << "}}"); + } + _familyMap = familyMapPtr; _lastUpdateTime = qChunkMap.updateTime; diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index f3d9365327..a1b7df30a4 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -205,7 +205,7 @@ class CzarChunkMap { /// @throws ChunkMapException void verify(std::string const& familyName) const; - static std::string dumpChunkMap(ChunkMap const& chunkMap); + std::string dumpChunkMap() const; static std::string dumpWorkerChunkMap(WorkerChunkMap const& wcMap); diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index e81b0e168c..42369927e2 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -86,7 +86,7 @@ void CzarRegistry::_registryUpdateLoop() { {{"name", _czarConfig->name()}, {"id", _czarConfig->id()}, {"management-port", _czarConfig->replicationHttpPort()}, - {"management-host-name", util::get_current_host_fqdn()}}}}); + {"management-host-name", util::getCurrentHostFqdnBlocking()}}}}); string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; LOGS(_log, LOG_LVL_TRACE, __func__ << " czarPost url=" << url << " request=" << request.dump() << " headers=" << headers[0]); @@ -113,7 +113,7 @@ void CzarRegistry::_registryWorkerInfoLoop() { string const replicationInstanceId = _czarConfig->replicationInstanceId(); string const replicationAuthKey = _czarConfig->replicationAuthKey(); uint64_t const czarStartTime = Czar::czarStartupTime; - + string const fqdn = util::getCurrentHostFqdnBlocking(); vector const headers; auto const method = http::Method::GET; string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + @@ -133,9 +133,9 @@ void CzarRegistry::_registryWorkerInfoLoop() { protojson::WorkerContactInfo::WCMapPtr wMap = _buildMapFromJson(response); // Update the values in the map { - auto czInfo = protojson::CzarContactInfo::create( - _czarConfig->name(), _czarConfig->id(), _czarConfig->replicationHttpPort(), - util::get_current_host_fqdn(), czarStartTime); + auto czInfo = protojson::CzarContactInfo::create(_czarConfig->name(), _czarConfig->id(), + _czarConfig->replicationHttpPort(), fqdn, + czarStartTime); lock_guard lck(_cmapMtx); if (wMap != nullptr) { _contactMap = wMap; diff --git a/src/czar/HttpCzarSvc.cc b/src/czar/HttpCzarSvc.cc index 23d36d396b..401f293d12 100644 --- a/src/czar/HttpCzarSvc.cc +++ b/src/czar/HttpCzarSvc.cc @@ -192,4 +192,14 @@ void HttpCzarSvc::_registerHandlers() { }); } +string HttpCzarConfig::dump() const { + stringstream os; + os << "HttpCzarConfig{port=" << port << " numThreads=" << numThreads + << " numWorkerIngestThreads=" << numWorkerIngestThreads << " sslCertFile = ** sslPrivateKeyFile = **" + << " tmpDir=" << tmpDir << " maxQueuedRequests=" << maxQueuedRequests + << " clientConnPoolSize=" << clientConnPoolSize << " numBoostAsioThreads=" << numBoostAsioThreads + << "}"; + return os.str(); +}; + } // namespace lsst::qserv::czar diff --git a/src/czar/HttpCzarSvc.h b/src/czar/HttpCzarSvc.h index b3cb32c743..eec28f2542 100644 --- a/src/czar/HttpCzarSvc.h +++ b/src/czar/HttpCzarSvc.h @@ -60,6 +60,7 @@ struct HttpCzarConfig { std::size_t maxQueuedRequests = 0; ///< 0 implies unlimited std::size_t clientConnPoolSize = 0; ///< 0 implies the default set by libcurl std::size_t numBoostAsioThreads = 0; ///< 0 implies the number of hardware threads + std::string dump() const; }; /** diff --git a/src/czar/qserv-czar-http.cc b/src/czar/qserv-czar-http.cc index cc24095eae..edc0fb475a 100644 --- a/src/czar/qserv-czar-http.cc +++ b/src/czar/qserv-czar-http.cc @@ -33,6 +33,9 @@ // Third party headers #include "boost/program_options.hpp" +// LSST headers +#include "lsst/log/Log.h" + // Qserv headers #include "cconfig/CzarConfig.h" #include "czar/Czar.h" @@ -47,6 +50,7 @@ namespace qserv = lsst::qserv; namespace { char const* const help = "The HTTP-based Czar frontend."; char const* const context = "[CZAR-HTTP-FRONTEND]"; +LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.czarhttp"); } // namespace int main(int argc, char* argv[]) { @@ -131,21 +135,25 @@ int main(int argc, char* argv[]) { return 0; } bool const verbose = vm.count("verbose") > 0; + + std::stringstream os; + os << ::context << " Czar name: " << czarName << "\n" + << ::context << " Configuration file: " << configFilePath << "\n" + << ::context << " Port: " << httpCzarConfig.port << "\n" + << ::context << " Number of threads: " << httpCzarConfig.numThreads << "\n" + << ::context << " Number of worker ingest threads: " << httpCzarConfig.numWorkerIngestThreads << "\n" + << ::context << " SSL certificate file: " << httpCzarConfig.sslCertFile << "\n" + << ::context << " SSL private key file: " << httpCzarConfig.sslPrivateKeyFile << "\n" + << ::context << " Temporary directory: " << httpCzarConfig.tmpDir << "\n" + << ::context << " Max.number of queued requests: " << httpCzarConfig.maxQueuedRequests << "\n" + << ::context << " Connection pool size (libcurl): " << httpCzarConfig.clientConnPoolSize << "\n" + << ::context << " Number of BOOST ASIO threads: " << httpCzarConfig.numBoostAsioThreads << "\n" + << ::context << " HTTP user: " << httpUser << "\n" + << ::context << " HTTP password: ******" << endl; + LOGS(_log, LOG_LVL_INFO, " czar-http startup " << os.str()); + if (verbose) { - cout << ::context << " Czar name: " << czarName << "\n" - << ::context << " Configuration file: " << configFilePath << "\n" - << ::context << " Port: " << httpCzarConfig.port << "\n" - << ::context << " Number of threads: " << httpCzarConfig.numThreads << "\n" - << ::context << " Number of worker ingest threads: " << httpCzarConfig.numWorkerIngestThreads - << "\n" - << ::context << " SSL certificate file: " << httpCzarConfig.sslCertFile << "\n" - << ::context << " SSL private key file: " << httpCzarConfig.sslPrivateKeyFile << "\n" - << ::context << " Temporary directory: " << httpCzarConfig.tmpDir << "\n" - << ::context << " Max.number of queued requests: " << httpCzarConfig.maxQueuedRequests << "\n" - << ::context << " Connection pool size (libcurl): " << httpCzarConfig.clientConnPoolSize << "\n" - << ::context << " Number of BOOST ASIO threads: " << httpCzarConfig.numBoostAsioThreads << "\n" - << ::context << " HTTP user: " << httpUser << "\n" - << ::context << " HTTP password: ******" << endl; + cout << os.str(); } try { auto const czar = czar::Czar::createCzar(configFilePath, czarName); diff --git a/src/http/BaseModule.h b/src/http/BaseModule.h index 686f3aef9b..61208461ae 100644 --- a/src/http/BaseModule.h +++ b/src/http/BaseModule.h @@ -219,7 +219,7 @@ class BaseModule { */ void sendData(nlohmann::json& result); - std::string authKey() const { return _authKey; } + AuthContext getAuthContext() const { return _authContext; } private: /** diff --git a/src/mysql/CMakeLists.txt b/src/mysql/CMakeLists.txt index 23e418d931..6aa2007a5b 100644 --- a/src/mysql/CMakeLists.txt +++ b/src/mysql/CMakeLists.txt @@ -2,6 +2,7 @@ add_library(mysql SHARED) target_sources(mysql PRIVATE CsvBuffer.cc + CsvMemDisk.cc LocalInfile.cc MySqlConfig.cc MySqlConnection.cc diff --git a/src/mysql/CsvMemDisk.cc b/src/mysql/CsvMemDisk.cc new file mode 100644 index 0000000000..4fc0502e6d --- /dev/null +++ b/src/mysql/CsvMemDisk.cc @@ -0,0 +1,249 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2014-2015 AURA/LSST. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "mysql/CsvMemDisk.h" + +// System headers +#include +#include +#include +#include +#include +#include + +// Third-party headers +#include + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "mysql/LocalInfileError.h" +#include "mysql/MySqlUtils.h" +#include "util/Bug.h" + +using namespace std; +namespace sfs = std::filesystem; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.mysql.CsvMemDisk"); +} // namespace + +namespace lsst::qserv::mysql { + +TransferTracker::Ptr TransferTracker::_globalMt; + +void TransferTracker::setup(std::size_t max, string const& directory, std::size_t minMBInMem, + std::size_t maxResultTableSizeBytes, CzarIdType czarId) { + if (_globalMt != nullptr) { + throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); + } + _globalMt = TransferTracker::Ptr(new TransferTracker(max, directory, minMBInMem, czarId)); +} + +bool TransferTracker::verifyDir(string const& dirName) { + sfs::path dir = dirName; + if (!(sfs::exists(dir) && sfs::is_directory(dir))) { + LOGS(_log, LOG_LVL_ERROR, "verifyDir, " + dirName + " is not a valid directory"); + return false; + } + return true; +} + +TransferTracker::MemoryRaii::Ptr TransferTracker::createRaii(size_t fileSize) { + MemoryRaii::Ptr pRaii(new MemoryRaii(fileSize)); + return pRaii; +} + +void TransferTracker::_incrTotal(size_t sz) { + lock_guard ulck(_mtx); + _total += sz; +} + +void TransferTracker::_decrTotal(size_t sz) { + lock_guard ulck(_mtx); + if (sz > _total) { + throw util::Bug(ERR_LOC, + "MemoryTracker::_decrTotal sz=" + to_string(sz) + " > total=" + to_string(_total)); + } + _total -= sz; +} + +CsvMemDisk::CsvMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId) + : _expectedBytes(expectedBytes), _qId(qId), _ujId(ujId) { + auto memTrack = TransferTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMemDisk constructor MemoryTracker is NULL"); + } + sfs::path fPath = memTrack->getDirectory(); + string fileName = memTrack->getBaseFileName() + "_" + to_string(memTrack->getCzarId()) + "_" + + to_string(_qId) + "_" + to_string(ujId); + fPath /= fileName; + _filePath = fPath; + _minBytesInMem = memTrack->getMinBytesInMem(); +} + +void CsvMemDisk::transferDataFromWorker(std::function transferFunc) { + auto memTrack = TransferTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMemDisk::waitReadyToRead MemoryTracker is NULL"); + } + _memRaii = memTrack->createRaii(_expectedBytes); + transferFunc(); +} + +bool CsvMemDisk::_mustWriteToTmpFile() { + // Once writing to file, this instance must keep writing to file. + if (_writingToTmpFile) return true; + + auto memTrack = TransferTracker::get(); + // If too much memory is being used for transfers, start writing large transfers to files. + if (memTrack->getTotal() > memTrack->getMax()) { + if (_records.size() > _minRecordsSize && _bytesRead > _minBytesInMem) { + _writingToTmpFile = true; + } + } + return _writingToTmpFile; +} + +void CsvMemDisk::push(char const* data, size_t size) { + // Push is always ok, no need to wait. + if (_cancelled) return; + _bytesRead += size; + if (_mustWriteToTmpFile()) { + _writeToTmpfile(data, size); + return; + } + if (data != nullptr && size != 0) { + _records.emplace_back(make_shared(data, size)); + } else { + // Empty string is meant to indicate the end of the stream. + _records.emplace_back(make_shared()); + } +} + +shared_ptr CsvMemDisk::pop() { + if (_records.size() > 0) { + shared_ptr front = _records.front(); + _records.pop_front(); + return front; + } + return _readFromTmpFile(); +} + +void CsvMemDisk::_writeToTmpfile(char const* data, std::size_t size) { + // Open the file if needed + auto oldState = _fState.exchange(OPEN_W); + if (oldState == INIT) { + _file.open(_filePath, fstream::out); + } + if (!_file.is_open() || _fState != OPEN_W) { + LOGS(_log, LOG_LVL_ERROR, + "CsvStrMemDisk::_writeTofile file isn't open " << _filePath << " or bad state=" << _fState); + _fileError = true; + return; + } + + _file.write(data, size); + _bytesWrittenToTmp += size; +} + +std::shared_ptr CsvMemDisk::_readFromTmpFile() { + if (_fState == OPEN_W) { + _fState = CLOSE_W; + _file.close(); + } + auto oldState = _fState.exchange(OPEN_R); + if (oldState == CLOSE_W) { + _file.open(_filePath, fstream::in); + _bytesLeft = _bytesWrittenToTmp; + } + if (!_file.is_open() || _fState != OPEN_R) { + // This is extremely unlikely and means something has gone wrong with the file system. + // If something has gone wrong with the file system, a crash may be incoming. + if (!getContaminated()) + LOGS(_log, LOG_LVL_ERROR, + "CsvStrMemDisk::_readFromfile file isn't open " << _filePath << " or bad state=" << _fState); + _setContaminated(); + return make_shared("$"); + } + + std::size_t buffSz = std::min(1'000'000ul, _bytesLeft); + auto strPtr = make_shared(); + strPtr->resize(buffSz); + _file.read(strPtr->data(), buffSz); + _bytesLeft -= buffSz; + return strPtr; +} + +CsvMemDisk::~CsvMemDisk() { + if (_fState != INIT) { + LOGS(_log, LOG_LVL_INFO, "~CsvStrMemDisk() remove " << _filePath); + _file.close(); + std::remove(_filePath.c_str()); + } +} + +class CsvMemDiskBuffer : public CsvBuffer { +public: + explicit CsvMemDiskBuffer(shared_ptr const& csvMemDisk) : _csvMemDisk(csvMemDisk) {} + + ~CsvMemDiskBuffer() override = default; + + unsigned fetch(char* buffer, unsigned bufLen) override { + if (bufLen == 0) { + throw LocalInfileError("CsvStreamBuffer::fetch Can't fetch non-positive bytes"); + } + auto csvMd = _csvMemDisk.lock(); + if (csvMd == nullptr) return 0; + if (_str == nullptr) { + _str = csvMd->pop(); + _offset = 0; + } + if (_str->empty()) return 0; + if (_offset >= _str->size()) { + _str = csvMd->pop(); + _offset = 0; + if (_str->empty()) return 0; + } + unsigned const bytesToCopy = min(bufLen, static_cast(_str->size() - _offset)); + ::memcpy(buffer, _str->data() + _offset, bytesToCopy); + _offset += bytesToCopy; + csvMd->increaseBytesFetched(bytesToCopy); + return bytesToCopy; + } + + string dump() const override { return "CsvStreamBuffer"; } + +private: + weak_ptr _csvMemDisk; + shared_ptr _str; + size_t _offset = 0; +}; + +shared_ptr newCsvMemDiskBuffer(shared_ptr const& csvMemDisk) { + return make_shared(csvMemDisk); +} + +} // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvMemDisk.h b/src/mysql/CsvMemDisk.h new file mode 100644 index 0000000000..d59f0e34d8 --- /dev/null +++ b/src/mysql/CsvMemDisk.h @@ -0,0 +1,214 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2015 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_MYSQL_CSVMEMDISK_H +#define LSST_QSERV_MYSQL_CSVMEMDISK_H + +// System headers +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +// Third-party headers +#include + +// qserv headers +#include "global/intTypes.h" +#include "mysql/CsvBuffer.h" + +namespace lsst::qserv::mysql { + +/// Track how much space is needed to store the current UberJob results while +/// transferring them from the workers and merging them to the result table. +/// How this effects the process depends on the TransferMethod. RAII methods +/// are used to ensure all allocations are freed. +/// MEMORYDISK - Instead new transfers waiting for memory to be +/// freed, most of the data will be written to disk when `_max` is +/// reached. The current plan is, per UberJob, to write create a +/// few CsvBuffers as is done now, and then write everything to +/// disk, and have pop read off disk when it runs out of existing +/// CsvBuffers. UberJobs with reasonable result sizes should be +/// unaffected. +/// @see CsvStrMemDisk +class TransferTracker { +public: + using Ptr = std::shared_ptr; + + TransferTracker() = delete; + + static bool verifyDir(std::string const& dirName); + static std::string getBaseFileName() { return std::string("qservtransfer"); } + + /// This class makes certain that any memory added to MemoryTracker + /// is removed from MemoryTracker. + class MemoryRaii { + public: + using Ptr = std::shared_ptr; + MemoryRaii() = delete; + ~MemoryRaii() { _globalMt->_decrTotal(memSize); } + + size_t const memSize; + friend class TransferTracker; + + private: + /// Only to be called by createRaii(), which locks the mutex. + explicit MemoryRaii(size_t memSize_) : memSize(memSize_) { _globalMt->_incrTotal(memSize); } + }; + friend class MemoryRaii; + + static void setup(std::size_t max, std::string const& directory, std::size_t minBytesInMem, + std::size_t maxResultTableSizeBytes, CzarIdType czarId); + static Ptr get() { return _globalMt; } + + /// Create a MemoryRaii instance to track `fileSize` bytes, and wait for free memory if `wait` is true. + MemoryRaii::Ptr createRaii(size_t fileSize); + + size_t getTotal() const { + std::lock_guard lg(_mtx); + return _total; + } + + std::size_t getMax() const { return _max; } + std::string getDirectory() const { return _directory; } + std::size_t getMinBytesInMem() const { return _minBytesInMem; } + CzarIdType getCzarId() const { return _czarId; } + +private: + TransferTracker(std::size_t max, std::string const& directory, std::size_t minBytesInMem, + CzarIdType czarId) + : _max(max), _directory(directory), _minBytesInMem(minBytesInMem), _czarId(czarId) {} + + /// This function only to be called via createRaii. + void _incrTotal(size_t sz); + + /// This function only to be called by ~MemoryRaii() + void _decrTotal(size_t sz); + + static Ptr _globalMt; + + mutable std::mutex _mtx; + std::size_t _total = 0; + std::size_t const _max; + std::string const _directory; + std::size_t const _minBytesInMem; + CzarIdType const _czarId; +}; + +/// Store transfer data in memory until too much memory is being used. +/// By setting the maximum acceptable amount of memory to 0, this +/// effectively becomes writing results to disk. +/// Collecting data from the worker, writing it to disk, reading +/// it back, and merging is expected to be linear, run within a +/// single thread. +/// The intention is that most reasonable size requests can be handled +/// within memory, which is highly likely to be the fastest method. +/// If a lot of memory (more than TransferTraker::_max) is being used by +/// all current transfers, then transfers greater than _minBytesInMem +/// will be written to disk until memory is free. +/// If _contaminated or _fileError get set to true, there are probably +/// catastrophic file system problems. +class CsvMemDisk { +public: + enum FileState { INIT, OPEN_W, CLOSE_W, OPEN_R, CLOSED }; + + static std::shared_ptr create(std::size_t expectedBytes, QueryId qId, UberJobId ujId) { + return std::shared_ptr(new CsvMemDisk(expectedBytes, qId, ujId)); + } + + CsvMemDisk() = delete; + CsvMemDisk(CsvMemDisk const&) = delete; + CsvMemDisk& operator=(CsvMemDisk const&) = delete; + ~CsvMemDisk(); + + void push(char const* data, std::size_t size); + + std::shared_ptr pop(); + + /// This version never waits. + void transferDataFromWorker(std::function transferFunc); + + /// True if a file error happened before results would be contaminated. + bool isFileError() const { return _fileError; } + + /// Stop transferring data before if the query has been cancelled. + void cancel() { _cancelled = true; } + + /// Indicates there was a file system error and the file could not be opened. + bool getContaminated() const { return _contaminated; } + + void increaseBytesFetched(size_t bytesToCopy) { _bytesFetched += bytesToCopy; } + size_t getBytesFetched() const { return _bytesFetched; } + +private: + CsvMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId); + + void _writeToTmpfile(char const* data, std::size_t size); + + /// Read from the file, which should only happen after all writing has finished. + std::shared_ptr _readFromTmpFile(); + + bool _mustWriteToTmpFile(); + + void _setContaminated() { _contaminated = true; } + + std::atomic _cancelled = false; + size_t _bytesFetched = 0; + std::list> _records; + + size_t _bytesRead = 0; + size_t const _expectedBytes; + + /// Indicates there was a file system error and the file could not be opened. + bool _contaminated = false; + + /// Have at least on record ready to be pushed + unsigned int const _minRecordsSize = 1; + std::size_t _minBytesInMem; + + bool _writingToTmpFile = false; + std::string const _directory; + std::string const _baseName; + QueryId const _qId; + UberJobId const _ujId; + + std::atomic _fState = INIT; + std::string _filePath; ///< file path, constant once set. + std::fstream _file; + + bool _fileError = false; + std::size_t _bytesWrittenToTmp = 0; + std::size_t _bytesLeft = 0; + + TransferTracker::MemoryRaii::Ptr _memRaii; +}; + +std::shared_ptr newCsvMemDiskBuffer(std::shared_ptr const& csvMemDisk); + +} // namespace lsst::qserv::mysql +#endif // LSST_QSERV_MYSQL_CSVMEMDISK_H diff --git a/src/mysql/LocalInfile.cc b/src/mysql/LocalInfile.cc index 66c06a2f16..e999996267 100644 --- a/src/mysql/LocalInfile.cc +++ b/src/mysql/LocalInfile.cc @@ -154,11 +154,7 @@ std::string LocalInfile::Mgr::prepareSrc(std::shared_ptr const& csvBu int LocalInfile::Mgr::local_infile_init(void** ptr, const char* filename, void* userdata) { assert(userdata); LocalInfile::Mgr* m = static_cast(userdata); -<<<<<<< HEAD - auto csvBuffer = m->get(std::string(filename)); -======= auto csvBuffer = m->getCsv(std::string(filename)); ->>>>>>> 24f5d37ea (Added JobErrorMsg.) assert(csvBuffer); LocalInfile* lf = new LocalInfile(filename, csvBuffer); *ptr = lf; @@ -192,17 +188,10 @@ void LocalInfile::Mgr::setBuffer(std::string const& filename, std::shared_ptr LocalInfile::Mgr::get(std::string const& filename) { - std::lock_guard lock(_mapMutex); - CsvBufferMap::iterator i = _map.find(filename); - if (i == _map.end()) { -======= std::shared_ptr LocalInfile::Mgr::getCsv(std::string const& filename) { std::lock_guard lock(_mapMutex); auto i = _mapCsv.find(filename); if (i == _mapCsv.end()) { ->>>>>>> 24f5d37ea (Added JobErrorMsg.) return std::shared_ptr(); } return i->second; @@ -218,11 +207,7 @@ std::string LocalInfile::Mgr::_nextFilename() { bool LocalInfile::Mgr::_set(std::string const& filename, std::shared_ptr const& csvBuffer) { std::lock_guard lock(_mapMutex); -<<<<<<< HEAD - auto res = _map.insert(std::pair>(filename, csvBuffer)); -======= auto res = _mapCsv.insert(std::pair>(filename, csvBuffer)); ->>>>>>> 24f5d37ea (Added JobErrorMsg.) return res.second; } diff --git a/src/protojson/testUberJobMsg.cc b/src/protojson/testUberJobMsg.cc index 95f8d8627f..a16a047b7c 100644 --- a/src/protojson/testUberJobMsg.cc +++ b/src/protojson/testUberJobMsg.cc @@ -49,13 +49,13 @@ BOOST_AUTO_TEST_SUITE(Suite) string testA() { string ta = - R"({"maxtablesizemb":5432,"scaninteractive":true,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":39,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; + R"({"maxtablesizemb":5432,"scaninteractive":true,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":51,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; return ta; } string testB() { string tb = - R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"scaninteractive":false,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":39,"worker":"db04"})"; + R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"scaninteractive":false,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":51,"worker":"db04"})"; return tb; } diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 57ef5e3161..076df78e9e 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -91,8 +91,7 @@ namespace lsst::qserv::qdisp { // class Executive implementation //////////////////////////////////////////////////////////////////////// Executive::Executive(int secondsBetweenUpdates, shared_ptr const& ms, - util::QdispPool::Ptr const& qdispPool, - shared_ptr const& queryProgress, + util::QdispPool::Ptr const& qdispPool, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, shared_ptr const& querySession) : _messageStore(ms), @@ -124,18 +123,17 @@ Executive::~Executive() { } } } - qdisp::CzarStats::get()->untrackQueryProgress(_id); } - Executive::Ptr Executive::create(int secsBetweenUpdates, shared_ptr const& ms, - std::shared_ptr const& qdispPool, + shared_ptr const& qdispPool, shared_ptr const& queryProgress, shared_ptr const& queryProgressHistory, shared_ptr const& querySession, boost::asio::io_service& asioIoService) { LOGS(_log, LOG_LVL_DEBUG, "Executive::" << __func__); - Executive::Ptr ptr(new Executive(secsBetweenUpdates, ms, qdispPool, queryProgress, queryProgressHistory, querySession)); + Executive::Ptr ptr(new Executive(secsBetweenUpdates, ms, qdispPool, queryProgress, queryProgressHistory, + querySession)); // Start the query progress monitoring timer (if enabled). The query status // will be sampled on each expiration event of the timer. Note that the timer @@ -784,15 +782,17 @@ void Executive::checkLimitRowComplete() { } void Executive::checkResultFileSize(uint64_t fileSize) { + if (_cancelled || isRowLimitComplete()) return; _totalResultFileSize += fileSize; - if (_cancelled) return; size_t const MB_SIZE_BYTES = 1024 * 1024; uint64_t maxResultTableSizeBytes = cconfig::CzarConfig::instance()->getMaxTableSizeMB() * MB_SIZE_BYTES; LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " sz=" << fileSize << " total=" << _totalResultFileSize << " max=" << maxResultTableSizeBytes); - if (_totalResultFileSize > maxResultTableSizeBytes) { + + if ((fileSize > maxResultTableSizeBytes) || + (!_limitSquashApplies && _totalResultFileSize > maxResultTableSizeBytes)) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " total=" << _totalResultFileSize << " max=" << maxResultTableSizeBytes); // _totalResultFileSize may include non zero values from dead UberJobs, @@ -812,11 +812,61 @@ void Executive::checkResultFileSize(uint64_t fileSize) { util::Error err(util::ErrorCode::CZAR_RESULT_TOO_LARGE, "Incomplete result already too large " + to_string(total)); _multiError.push_back(err); + _resultFileSizeExceeded = true; squash("czar, file too large"); } } } +shared_ptr> Executive::getLimitSquashLock() { + shared_ptr> ptr(new lock_guard(_mtxLimitSquash)); + return ptr; +} + +void Executive::collectFile(std::shared_ptr ujPtr, std::string const& fileUrl, uint64_t fileSize, + uint64_t rowCount, std::string const& idStr) { + // Limit collecting LIMIT queries to one at a time, but only those. + shared_ptr> limitSquashL; + if (_limitSquashApplies) { + limitSquashL.reset(new lock_guard(_mtxLimitSquash)); + } + MergeEndStatus flushStatus = ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize); + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << "ujId=" << ujPtr->getUjId() << " success=" << flushStatus.success + << " contaminated=" << flushStatus.contaminated); + if (flushStatus.success) { + CzarStats::get()->addTotalRowsRecv(rowCount); + CzarStats::get()->addTotalBytesRecv(fileSize); + } else { + if (flushStatus.contaminated) { + // This would probably indicate malformed file+rowCount or writing the result table failed. + // If any merging happened, the result table is ruined. + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << "ujId=" << ujPtr->getUjId() + << " flushHttp failed after merging, results ruined."); + } else { + // Perhaps something went wrong with file collection, so it is worth trying the jobs again + // by abandoning this UberJob. + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << "ujId=" << ujPtr->getUjId() << " flushHttp failed, retrying Jobs."); + } + ujPtr->importResultError(flushStatus.contaminated, "mergeError", "merging failed"); + } + + // At this point all data for this job have been read and merged + bool const statusSet = ujPtr->importResultFinish(); + if (!statusSet) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << "ujId=" << ujPtr->getUjId() << " failed to set status, squashing " + << getIdStr()); + // Something has gone very wrong + squash(cName(__func__) + " couldn't set UberJob status"); + return; + } + addResultRows(rowCount); + checkLimitRowComplete(); +} + ostream& operator<<(ostream& os, Executive::JobMap::value_type const& v) { auto const& status = v.second->getStatus(); os << v.first << ": " << *status; diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 969b56f866..5e89098aef 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -50,41 +50,32 @@ #include "util/threadSafe.h" #include "util/ThreadPool.h" -namespace lsst::qserv::qmeta { -class QProgress; -class QProgressHistory; -} // namespace lsst::qserv::qmeta - - -namespace lsst::qserv { - -namespace ccontrol { +namespace lsst::qserv::ccontrol { class UserQuerySelect; } -namespace qmeta { +namespace lsst::qserv::qmeta { class MessageStore; -} // namespace qmeta +class QProgress; +class QProgressHistory; +} // namespace lsst::qserv::qmeta -namespace qproc { +namespace lsst::qserv::qproc { class QuerySession; } // namespace lsst::qserv::qproc -namespace qdisp { -class JobQuery; -class UberJob; -} // namespace qdisp - - -namespace util { +namespace lsst::qserv::util { class AsyncTimer; class PriorityCommand; class QdispPool; -} // namespace util +} // namespace lsst::qserv::util // This header declarations namespace lsst::qserv::qdisp { +class JobQuery; +class UberJob; + /// class Executive manages the execution of jobs for a UserQuery. class Executive : public std::enable_shared_from_this { public: @@ -233,6 +224,15 @@ class Executive : public std::enable_shared_from_this { /// cancel this user query. void checkResultFileSize(uint64_t fileSize = 0); + /// Returns a pointer to a lock on _mtxLimitSquash. + std::shared_ptr> getLimitSquashLock(); + + void collectFile(std::shared_ptr ujPtr, std::string const& fileUrl, uint64_t fileSize, + uint64_t rowCount, std::string const& idStr); + + /// Return true if the result size limit has been exceeded. + bool resultSizeLimitExceeded() const { return _resultFileSizeExceeded; } + protected: Executive(int secondsBetweenUpdates, std::shared_ptr const& ms, std::shared_ptr const& sharedResources, @@ -273,8 +273,9 @@ class Executive : public std::enable_shared_from_this { /// How many jobs are used in this query. 1 avoids possible 0 of 0 jobs completed race condition. /// The correct value is set when it is available. std::atomic _totalJobs{1}; - std::shared_ptr - _qdispPool; ///< Shared thread pool for handling commands to and from workers. + + /// Shared thread pool for handling commands to and from workers. + std::shared_ptr _qdispPool; std::deque> _jobStartCmdList; ///< list of jobs to start. @@ -351,6 +352,15 @@ class Executive : public std::enable_shared_from_this { std::atomic _totalResultFileSize{0}; ///< Total size of all UberJob result files. std::atomic _jobCancelCount{0}; ///< Total number of JOB_CANCEL messages received. + + /// This mutex is used to limit collecting result files to one at a time + /// but only when the executive will squash the query when the limit is reached. + /// This keeps data transfers (and temporary storage requirements) from + /// getting out of hand. + std::mutex _mtxLimitSquash; + + /// Set to true if the result file is too large. + std::atomic _resultFileSizeExceeded{false}; }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index ad6fd11eba..e238d4d692 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -66,11 +66,6 @@ class ResponseHandler { void setUberJob(std::weak_ptr const& ujPtr) { _uberJob = ujPtr; } virtual ~ResponseHandler() {} - /// Process a request for pulling and merging a job result into the result table - /// @param responseSummary - worker response to be analyzed and processed - /// @return true if successful (no error) - virtual bool flush(proto::ResponseSummary const& responseSummary) = 0; - /// Collect result data from the worker and merge it with the query result table. /// If MergeEndStatus.success == true, then everything is fine. /// If not .success, and not .contaminated, the user query can be saved by abandoning diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 5413b97fa2..302c9e9858 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -27,7 +27,6 @@ #include // Third-party headers -#include #include "nlohmann/json.hpp" // Qserv headers @@ -127,7 +126,7 @@ void UberJob::runUberJob() { uint64_t maxTableSizeMB = czarConfig->getMaxTableSizeMB(); auto czInfo = protojson::CzarContactInfo::create( czarConfig->name(), czarConfig->id(), czarConfig->replicationHttpPort(), - util::get_current_host_fqdn(), czar::Czar::czarStartupTime); + czar::Czar::getCzar()->getFqdn(), czar::Czar::czarStartupTime); auto scanInfoPtr = exec->getScanInfo(); bool scanInteractive = exec->getScanInteractive(); @@ -269,12 +268,12 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ auto exec = _executive.lock(); if (exec == nullptr) { LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive"); - return _importResultError(true, "cancelled", "Query cancelled - no executive"); + return importResultError(true, "cancelled", "Query cancelled - no executive"); } if (exec->getCancelled()) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); - return _importResultError(true, "cancelled", "Query cancelled"); + return importResultError(true, "cancelled", "Query cancelled"); } if (exec->isRowLimitComplete()) { @@ -283,7 +282,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_INFO, "UberJob ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } - return _importResultError(false, "rowLimited", "Enough rows already"); + return importResultError(false, "rowLimited", "Enough rows already"); } LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " fileSize=" << fileSize); @@ -291,7 +290,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); if (!statusSet) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " setStatusFail could not set status to RESPONSE_READY"); - return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); + return importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); } weak_ptr ujThis = weak_from_this(); @@ -305,31 +304,14 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); return; } - MergeEndStatus flushStatus = ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize); - LOGS(_log, LOG_LVL_TRACE, - ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushStatus.success - << " contaminated=" << flushStatus.contaminated); - if (flushStatus.success) { - qdisp::CzarStats::get()->addTotalRowsRecv(rowCount); - qdisp::CzarStats::get()->addTotalBytesRecv(fileSize); - } else { - if (flushStatus.contaminated) { - // This would probably indicate malformed file+rowCount or writing the result table failed. - // If any merging happened, the result table is ruined. - LOGS(_log, LOG_LVL_ERROR, - ujPtr->cName(__func__) - << "::fileCollectFunc flushHttp failed after merging, results ruined."); - } else { - // Perhaps something went wrong with file collection, so it is worth trying the jobs again - // by abandoning this UberJob. - LOGS(_log, LOG_LVL_ERROR, - ujPtr->cName(__func__) << "::fileCollectFunc flushHttp failed, retrying Jobs."); - } - ujPtr->_importResultError(flushStatus.contaminated, "mergeError", "merging failed"); + auto exec = ujPtr->getExecutive(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, + "UberJob::fileCollectFunction exec ptr is null " << idStr << " " << fileUrl); + return; } - // At this point all data for this job have been read. - ujPtr->_importResultFinish(rowCount); + exec->collectFile(ujPtr, fileUrl, fileSize, rowCount, idStr); }; auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); @@ -388,7 +370,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { return _workerErrorFinish(deleteData, errType, ""); } -json UberJob::_importResultError(bool shouldCancel, string const& errorType, string const& note) { +json UberJob::importResultError(bool shouldCancel, string const& errorType, string const& note) { json jsRet = {{"success", 0}, {"errortype", errorType}, {"note", note}}; // In all cases, the worker should delete the file as this czar will not ask for it. @@ -419,32 +401,20 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str return jsRet; } -void UberJob::_importResultFinish(uint64_t resultRows) { +bool UberJob::importResultFinish() { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); - return; - } - /// If this is called, the file has been collected and the worker should delete it /// /// This function should call markComplete for all jobs in the uberjob /// and return a "success:1" json message to be sent to the worker. bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); - if (!statusSet) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to set status, squashing " << getIdStr()); - // Something has gone very wrong - exec->squash("UberJob::_importResultFinish couldn't set status"); - return; + if (statusSet) { + bool const success = true; + callMarkCompleteFunc(success); // sets status to COMPLETE } - - bool const success = true; - callMarkCompleteFunc(success); // sets status to COMPLETE - exec->addResultRows(resultRows); - exec->checkLimitRowComplete(); + return statusSet; } nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 313a4c7405..10fede3879 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -116,6 +116,14 @@ class UberJob : public std::enable_shared_from_this { void setResultFileSize(uint64_t fileSize) { _resultFileSize = fileSize; } uint64_t getResultFileSize() { return _resultFileSize; } + /// Update UberJob status, return true if successful. + bool importResultFinish(); + + /// Import and error from trying to collect results. + /// TODO:UJ The strings for errorType should have a centralized location in the code - global or util + nlohmann::json importResultError(bool shouldCancel, std::string const& errorType, + std::string const& note); + std::ostream& dumpOS(std::ostream& os) const; std::string dump() const; friend std::ostream& operator<<(std::ostream& os, UberJob const& uj); @@ -137,14 +145,6 @@ class UberJob : public std::enable_shared_from_this { /// reassignment. The list of _jobs is cleared, so multiple calls of this should be harmless. void _unassignJobs(); - /// Import and error from trying to collect results. - /// TODO:UJ The strings for errorType should have a centralized location in the code - global or util - nlohmann::json _importResultError(bool shouldCancel, std::string const& errorType, - std::string const& note); - - /// Let the executive know that all Jobs in UberJob are complete. - void _importResultFinish(uint64_t resultRows); - /// Let the Executive know about errors while handling results. nlohmann::json _workerErrorFinish(bool successful, std::string const& errorType = std::string(), std::string const& note = std::string()); @@ -160,9 +160,8 @@ class UberJob : public std::enable_shared_from_this { QueryId const _queryId; UberJobId const _uberJobId; qmeta::CzarId const _czarId; - int const _rowLimit; + int const _rowLimit; ///< Number of rows in the query LIMIT clause. uint64_t _resultFileSize = 0; - std::string const _idStr; // Map of workerData diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 9110048b74..f36a31caf8 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -41,9 +41,6 @@ #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" -#include "qdisp/QueryRequest.h" -#include "qdisp/SharedResources.h" -#include "qdisp/XrdSsiMocks.h" #include "qmeta/QProgress.h" #include "qmeta/QProgressHistory.h" #include "qmeta/MessageStore.h" @@ -144,9 +141,12 @@ class ExecutiveUT : public Executive { ~ExecutiveUT() override = default; ExecutiveUT(int qmetaTimeBetweenUpdates, shared_ptr const& ms, - util::QdispPool::Ptr const& qdispPool, shared_ptr const& qStatus, + util::QdispPool::Ptr const& qdispPool, shared_ptr const& qProgress, + shared_ptr const& queryProgressHistory, shared_ptr const& querySession, TestInfo::Ptr const& testInfo_) - : Executive(qmetaTimeBetweenUpdates, ms, qdispPool, qStatus, querySession), testInfo(testInfo_) {} + : Executive(qmetaTimeBetweenUpdates, ms, qdispPool, qProgress, queryProgressHistory, + querySession), + testInfo(testInfo_) {} void assignJobsToUberJobs() override { vector ujVect; @@ -269,7 +269,7 @@ class SetupTest { std::shared_ptr queryProgressHistory; // No updating QProgressHistory, nullptr ex = qdisp::ExecutiveUT::PtrUT(new qdisp::ExecutiveUT(60, ms, qdispPool, qProgress, - queryProgressHistory, nullptr, testInfo)); + queryProgressHistory, nullptr, testInfo)); LOGS(_log, LOG_LVL_INFO, "SetupTest end"); } ~SetupTest() {} diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index a1bf76119b..bba3ec3906 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -52,7 +52,6 @@ namespace { // Current version of QMeta schema char const VERSION_STR[] = "17"; - LOG_LOGGER _log = LOG_GET("lsst.qserv.qmeta.QMetaMysql"); using lsst::qserv::qmeta::QInfo; diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index d05ad55cce..cc5bcd8df4 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -56,7 +56,7 @@ // Qserv headers #include "cconfig/CzarConfig.h" #include "global/intTypes.h" -#include "mysql/CsvBuffer.h" +#include "mysql/CsvMemDisk.h" #include "qdisp/CzarStats.h" #include "qdisp/UberJob.h" #include "qproc/DatabaseModels.h" @@ -158,37 +158,13 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { _queryIdStrSet = true; } -void InfileMerger::mergeCompleteFor(int jobId) { - std::lock_guard resultSzLock(_mtxResultSizeMtx); - _totalResultSize += _perJobResultSize[jobId]; // TODO:UJ this can probably be simplified -} - bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSize, - std::shared_ptr const& csvStream) { - UberJobId const uJobId = uberJob->getUjId(); + std::shared_ptr const& csvMemDisk) { std::string queryIdJobStr = uberJob->getIdStr(); if (!_queryIdStrSet) { _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); } - // Check if the final result size is too large. It should be safe to do this - // here as the only expected errors at this point are failures in transmission. - // Even if there is a failure in transmission, the retry would be expected - // to put the result size over the limit again. - { - lock_guard resultSzLock(_mtxResultSizeMtx); - _perJobResultSize[uJobId] += fileSize; - size_t tResultSize = _totalResultSize + _perJobResultSize[uJobId]; - if (tResultSize > _maxResultTableSizeBytes) { - string str = queryIdJobStr + " cancelling the query, queryResult table " + _mergeTable + - " is too large at " + to_string(tResultSize) + " bytes, max allowed size is " + - to_string(_maxResultTableSizeBytes) + " bytes"; - LOGS(_log, LOG_LVL_ERROR, str); - _error = util::Error(-1, str, -1); - return false; - } - } - TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, bool success) { if (!success) return; @@ -202,7 +178,7 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi // Add columns to rows in virtFile. util::Timer virtFileT; virtFileT.start(); - auto const csvBuffer = mysql::newCsvStreamBuffer(csvStream); + auto const csvBuffer = mysql::newCsvMemDiskBuffer(csvMemDisk); std::string const virtFile = _infileMgr.prepareSrc(csvBuffer); std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); virtFileT.stop(); @@ -217,19 +193,41 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi return true; } + // Need to block here to make sure the result able needs these rows or not. + lock_guard lgFinal(_finalMergeMtx); // Don't merge if the query got cancelled. auto executive = uberJob->getExecutive(); if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { - csvStream->cancel(); // After this point, the file has to be read return true; } + if (csvMemDisk->isFileError()) { + // The file couldn't be opened for writing, so giving up + // now should keep the result table from getting contaminated. + return false; + } + auto start = std::chrono::system_clock::now(); + // The following will call some version of CsvStream::pop() at least once. ret = _applyMysqlMyIsam(infileStatement, fileSize); auto end = std::chrono::system_clock::now(); auto mergeDur = std::chrono::duration_cast(end - start); LOGS(_log, LOG_LVL_DEBUG, "mergeDur=" << mergeDur.count()); - if (not ret) { + if (ret) { + lock_guard resultSzLock(_mtxResultSizeMtx); + _totalResultSize += fileSize; + size_t tResultSize = _totalResultSize; + /// Check file size here to see if it has gotten too large, this will probably only trip in LIMIT + /// queries. + if (tResultSize > _maxResultTableSizeBytes) { + string str = queryIdJobStr + " cancelling the query, queryResult table " + _mergeTable + + " is too large at " + to_string(tResultSize) + " bytes, max allowed size is " + + to_string(_maxResultTableSizeBytes) + " bytes"; + LOGS(_log, LOG_LVL_ERROR, str); + _error = util::Error(-1, str, -1); + return false; + } + } else { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); } LOGS(_log, LOG_LVL_TRACE, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); @@ -277,6 +275,7 @@ size_t InfileMerger::getTotalResultSize() const { return _totalResultSize; } bool InfileMerger::finalize(size_t& collectedBytes, int64_t& rowCount) { bool finalizeOk = true; collectedBytes = _totalResultSize; + lock_guard lgFinal(_finalMergeMtx); // block on other merges // TODO: Should check for error condition before continuing. if (_isFinished) { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::finalize(), but _isFinished == true"); diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index dd7e0be1ec..2a9179024e 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -46,7 +46,7 @@ // Forward declarations namespace lsst::qserv { namespace mysql { -class CsvStream; +class CsvMemDisk; class MysqlConfig; } // namespace mysql namespace qdisp { @@ -104,10 +104,7 @@ class InfileMerger { /// Merge the result data collected over Http. bool mergeHttp(std::shared_ptr const& uberJob, uint64_t fileSize, - std::shared_ptr const& csvStream); - - /// Indicate the merge for the job is complete. - void mergeCompleteFor(int jobId); + std::shared_ptr const& csvMemDisk); /// @return error details if finalize() returns false util::Error const& getError() const { return _error; } @@ -126,12 +123,6 @@ class InfileMerger { /// Check if the object has completed all processing. bool isFinished() const; - /// Check if the result size limit has been exceeded. - bool resultSizeLimitExceeded() const { return _resultSizeLimitExceeded.load(); } - - /// Check if the result size limit has been exceeded. - void setResultSizeLimitExceeded() { _resultSizeLimitExceeded.store(true); } - void setMergeStmtFromList(std::shared_ptr const& mergeStmt) const; /** @@ -171,19 +162,18 @@ class InfileMerger { bool _applySqlLocal(std::string const& sql, sql::SqlResults& results, sql::SqlErrorObject& errObj); bool _sqlConnect(sql::SqlErrorObject& errObj); - util::InstanceCount const _icIm{"InfileMerger"}; std::string _getQueryIdStr(); void _setQueryIdStr(std::string const& qIdStr); void _fixupTargetName(); bool _setupConnectionMyIsam(); - InfileMergerConfig _config; ///< Configuration - std::shared_ptr _sqlConn; ///< SQL connection - std::string _mergeTable; ///< Table for result loading - util::Error _error; ///< Error state - bool _isFinished = false; ///< Completed? - std::atomic _resultSizeLimitExceeded{false}; ///< Large result query? - std::mutex _sqlMutex; ///< Protection for SQL connection + InfileMergerConfig _config; ///< Configuration + std::shared_ptr _sqlConn; ///< SQL connection + std::string _mergeTable; ///< Table for result loading + util::Error _error; ///< Error state + bool _isFinished = false; ///< Completed? + std::mutex _sqlMutex; ///< Protection for SQL connection + mysql::MySqlConnection _mysqlConn; std::mutex _mysqlMutex; mysql::LocalInfile::Mgr _infileMgr; @@ -195,10 +185,10 @@ class InfileMerger { 10; ///< maximum number of times to retry connecting to the SQL database. /// Variable to track result size. Each - size_t const _maxResultTableSizeBytes; ///< Max result table size in bytes. - size_t _totalResultSize = 0; ///< Size of result so far in bytes. - std::map _perJobResultSize; ///< Result size for each job - std::mutex _mtxResultSizeMtx; ///< Protects _perJobResultSize and _totalResultSize. + size_t const _maxResultTableSizeBytes; ///< Max result table size in bytes. + size_t _totalResultSize = 0; ///< Size of result so far in bytes. + std::mutex _mtxResultSizeMtx; ///< Protects _totalResultSize. + std::mutex _finalMergeMtx; ///< Protects mysql result tables }; } // namespace lsst::qserv::rproc diff --git a/src/wcomms/HttpWorkerCzarModule.cc b/src/wcomms/HttpWorkerCzarModule.cc index 8a8ebd02f6..3c479ec12a 100644 --- a/src/wcomms/HttpWorkerCzarModule.cc +++ b/src/wcomms/HttpWorkerCzarModule.cc @@ -140,19 +140,19 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { } std::shared_ptr foremanPtr = foreman(); - std::string authKeyStr = authKey(); + auto authCtx = getAuthContext(); // It is important to create UberJobData at this point as it will be the only way to // inform the czar of errors after this function returns. auto ujData = wbase::UberJobData::create(ujId, ujCzInfo->czName, ujCzInfo->czId, ujCzInfo->czHostName, ujCzInfo->czPort, ujQueryId, ujRowLimit, maxTableSizeBytes, scanInfo, scanInteractive, targetWorkerId, foremanPtr, - authKeyStr, foremanPtr->httpPort()); + authCtx.authKey, foremanPtr->httpPort()); auto lFunc = [ujId, ujQueryId, ujCzInfo, ujRowLimit, maxTableSizeBytes, targetWorkerId, userQueryInfo, - uberJobMsg, foremanPtr, authKeyStr, ujData](util::CmdData*) { + uberJobMsg, foremanPtr, authCtx, ujData](util::CmdData*) { _buildTasks(ujId, ujQueryId, ujCzInfo, ujRowLimit, maxTableSizeBytes, targetWorkerId, - userQueryInfo, uberJobMsg, foremanPtr, authKeyStr, ujData); + userQueryInfo, uberJobMsg, foremanPtr, authCtx.authKey, ujData); }; util::Command::Ptr taskLoadCmd = std::make_shared(lFunc); diff --git a/src/wmain/WorkerMain.cc b/src/wmain/WorkerMain.cc index 1a306df378..bcab50f297 100644 --- a/src/wmain/WorkerMain.cc +++ b/src/wmain/WorkerMain.cc @@ -67,7 +67,6 @@ #include "wsched/GroupScheduler.h" #include "wsched/ScanScheduler.h" #include "wcomms/HttpSvc.h" -#include "wcomms/XrdName.h" using namespace lsst::qserv; using namespace nlohmann; @@ -97,46 +96,6 @@ std::shared_ptr makeChunkInventory(string const& worke return inventory; } -/** - * This function will keep periodically updating worker's info in the Replication - * System's Registry. - * @param id The unique identifier of a worker to be registered. - * @note The thread will terminate the process if the registraton request to the Registry - * was explicitly denied by the service. This means the application may be misconfigured. - * Transient communication errors when attempting to connect or send requests to - * the Registry will be posted into the log stream and ignored. - */ -void registryUpdateLoop(string const& id) { - auto const workerConfig = wconfig::WorkerConfig::instance(); - auto const method = http::Method::POST; - string const url = "http://" + workerConfig->replicationRegistryHost() + ":" + - to_string(workerConfig->replicationRegistryPort()) + "/qserv-worker"; - vector const headers = {"Content-Type: application/json"}; - json const request = json::object({{"version", http::MetaModule::version}, - {"instance_id", workerConfig->replicationInstanceId()}, - {"auth_key", workerConfig->replicationAuthKey()}, - {"worker", - {{"name", id}, - {"management-port", workerConfig->replicationHttpPort()}, - {"management-host-name", util::get_current_host_fqdn()}}}}); - string const requestContext = - "WorkerMain: '" + http::method2string(method) + "' request to '" + url + "'"; - http::Client client(method, url, request.dump(), headers); - while (true) { - try { - json const response = client.readAsJson(); - if (0 == response.at("success").get()) { - string const error = response.at("error").get(); - LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); - abort(); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); - } - this_thread::sleep_for(chrono::seconds(max(1U, workerConfig->replicationRegistryHearbeatIvalSec()))); - } -} - } // namespace namespace lsst::qserv::wmain { @@ -266,24 +225,70 @@ WorkerMain::WorkerMain() { // Begin periodically updating worker's status in the Replication System's registry // in the detached thread. This will continue before the application gets terminated. - thread registryUpdateThread(::registryUpdateLoop, _foreman->chunkInventory()->id()); - registryUpdateThread.detach(); + thread registryUpdateThread(&WorkerMain::_registryUpdateLoop, this); + _registryUpdateThread = move(registryUpdateThread); } void WorkerMain::waitForTerminate() { unique_lock uniq(_terminateMtx); - _terminateCv.wait(uniq, [this]() { return _terminate; }); + _terminateCv.wait(uniq, [this]() -> bool { return _terminate; }); } void WorkerMain::terminate() { - lock_guard lck(_terminateMtx); - _terminate = true; + { + lock_guard lck(_terminateMtx); + if (_terminate.exchange(true)) return; + ; + } _terminateCv.notify_all(); + _controlHttpSvc->stop(); } WorkerMain::~WorkerMain() { LOGS(_log, LOG_LVL_INFO, "WorkerMain shutdown."); - _controlHttpSvc->stop(); + terminate(); + _registryUpdateThread.join(); +} + +/** + * This function will keep periodically updating worker's info in the Replication + * System's Registry. + * @param id The unique identifier of a worker to be registered. + * @note The thread will terminate the process if the registraton request to the Registry + * was explicitly denied by the service. This means the application may be misconfigured. + * Transient communication errors when attempting to connect or send requests to + * the Registry will be posted into the log stream and ignored. + */ +void WorkerMain::_registryUpdateLoop() { + string const id = _foreman->chunkInventory()->id(); + auto const workerConfig = wconfig::WorkerConfig::instance(); + auto const method = http::Method::POST; + string const url = "http://" + workerConfig->replicationRegistryHost() + ":" + + to_string(workerConfig->replicationRegistryPort()) + "/qserv-worker"; + vector const headers = {"Content-Type: application/json"}; + json const request = json::object({{"version", http::MetaModule::version}, + {"instance_id", workerConfig->replicationInstanceId()}, + {"auth_key", workerConfig->replicationAuthKey()}, + {"worker", + {{"name", id}, + {"management-port", workerConfig->replicationHttpPort()}, + {"management-host-name", _foreman->getFqdn()}}}}); + string const requestContext = + "WorkerMain: '" + http::method2string(method) + "' request to '" + url + "'"; + http::Client client(method, url, request.dump(), headers); + while (!_terminate) { + try { + json const response = client.readAsJson(); + if (0 == response.at("success").get()) { + string const error = response.at("error").get(); + LOGS(_log, LOG_LVL_ERROR, requestContext + " was denied, error: '" + error + "'."); + abort(); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); + } + this_thread::sleep_for(chrono::seconds(max(1U, workerConfig->replicationRegistryHearbeatIvalSec()))); + } } } // namespace lsst::qserv::wmain diff --git a/src/wmain/WorkerMain.h b/src/wmain/WorkerMain.h index 7461b0936a..aca01ce893 100644 --- a/src/wmain/WorkerMain.h +++ b/src/wmain/WorkerMain.h @@ -59,12 +59,16 @@ class WorkerMain { std::string getName() const { return _name; } + /// End WorkerMain, calling this multiple times is harmless. void terminate(); void waitForTerminate(); private: WorkerMain(); + void _registryUpdateLoop(); + std::thread _registryUpdateThread; + /// Weak pointer to allow global access without complicating lifetime issues. static std::weak_ptr _globalWorkerMain; @@ -85,7 +89,7 @@ class WorkerMain { std::shared_ptr _controlHttpSvc; /// Set to true when the program should terminate. - bool _terminate = false; + std::atomic _terminate{false}; std::mutex _terminateMtx; std::condition_variable _terminateCv; }; From 2936e960988f41db3fc7f422f371eb18e255854e Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 18 Dec 2024 09:37:50 -0800 Subject: [PATCH 15/15] Added worker exe, memory/disk hybrid for transfering csv files, fixed tests. --- src/czar/ActiveWorker.cc | 2 +- src/protojson/ScanTableInfo.cc | 10 +- src/protojson/ScanTableInfo.h | 4 +- src/protojson/UberJobMsg.cc | 109 +-- src/protojson/UberJobMsg.h | 65 +- src/protojson/WorkerCzarComIssue.cc | 4 +- src/protojson/WorkerQueryStatusData.cc | 14 +- src/protojson/WorkerQueryStatusData.h | 8 +- src/protojson/testStatusData.cc | 22 +- src/protojson/testUberJobMsg.cc | 12 +- src/qdisp/Executive.cc | 1 - src/qdisp/UberJob.cc | 2 +- src/util/Command.cc | 21 + src/util/Command.h | 5 + src/util/InstanceCount.cc | 2 - src/wbase/FileChannelShared.cc | 1 + src/wbase/Task.cc | 110 +-- src/wbase/Task.h | 6 +- src/wbase/UberJobData.cc | 7 + src/wcomms/HttpWorkerCzarModule.h | 5 + src/wdb/CMakeLists.txt | 4 - src/wdb/QueryRunner.cc | 1 - src/wdb/testQueryRunner.cc | 256 ------- src/wmain/WorkerMain.cc | 1 - src/wpublish/QueriesAndChunks.cc | 14 +- src/wsched/BlendScheduler.cc | 5 + src/wsched/BlendScheduler.h | 2 + src/wsched/ChunkTasksQueue.cc | 1 + src/wsched/ScanScheduler.cc | 5 + src/wsched/ScanScheduler.h | 3 +- src/wsched/testSchedulers.cc | 965 +++++++++++++++---------- 31 files changed, 812 insertions(+), 855 deletions(-) delete mode 100644 src/wdb/testQueryRunner.cc diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index 12aae1fce6..6edf231ddc 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -139,7 +139,7 @@ void ActiveWorker::updateStateAndSendMessages(double timeoutAliveSecs, double ti { // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a // message to send to the worker. - jsWorkerReqPtr = _wqsData->serializeJson(maxLifetime); + jsWorkerReqPtr = _wqsData->toJson(maxLifetime); } // Always send the message as it's a way to inform the worker that this diff --git a/src/protojson/ScanTableInfo.cc b/src/protojson/ScanTableInfo.cc index 32da583bf2..c113a55289 100644 --- a/src/protojson/ScanTableInfo.cc +++ b/src/protojson/ScanTableInfo.cc @@ -98,7 +98,7 @@ void ScanInfo::sortTablesSlowestFirst() { std::sort(infoTables.begin(), infoTables.end(), func); } -nlohmann::json ScanInfo::serializeJson() const { +nlohmann::json ScanInfo::toJson() const { auto jsScanInfo = json({{"infoscanrating", scanRating}, {"infotables", json::array()}}); auto& jsInfoTables = jsScanInfo["infotables"]; @@ -127,6 +127,7 @@ ScanInfo::Ptr ScanInfo::createFromJson(nlohmann::json const& siJson) { auto lockInMem = http::RequestBodyJSON::required(jsElem, "silockinmem"); iTbls.emplace_back(db, table, lockInMem, sRating); } + siPtr->sortTablesSlowestFirst(); return siPtr; } @@ -137,8 +138,13 @@ std::ostream& operator<<(std::ostream& os, ScanTableInfo const& tbl) { return os; } +std::ostream& ScanInfo::dump(std::ostream& os) const { + os << "ScanInfo{speed=" << scanRating << " tables: " << util::printable(infoTables) << "}"; + return os; +} + std::ostream& operator<<(std::ostream& os, ScanInfo const& info) { - os << "ScanInfo{speed=" << info.scanRating << " tables: " << util::printable(info.infoTables) << "}"; + info.dump(os); return os; } diff --git a/src/protojson/ScanTableInfo.h b/src/protojson/ScanTableInfo.h index 061ea0c0f9..4f1b3e8ccf 100644 --- a/src/protojson/ScanTableInfo.h +++ b/src/protojson/ScanTableInfo.h @@ -71,13 +71,15 @@ class ScanInfo { static Ptr createFromJson(nlohmann::json const& ujJson); /// Return a json version of the contents of this class. - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; void sortTablesSlowestFirst(); int compareTables(ScanInfo const& rhs); ScanTableInfo::ListOf infoTables; int scanRating{Rating::FASTEST}; + + std::ostream& dump(std::ostream& os) const; }; std::ostream& operator<<(std::ostream& os, ScanTableInfo const& tbl); diff --git a/src/protojson/UberJobMsg.cc b/src/protojson/UberJobMsg.cc index e95560de09..e2a0f27156 100644 --- a/src/protojson/UberJobMsg.cc +++ b/src/protojson/UberJobMsg.cc @@ -70,25 +70,25 @@ UberJobMsg::UberJobMsg(unsigned int metaVersion, std::string const& replicationI } } -json UberJobMsg::serializeJson() const { +json UberJobMsg::toJson() const { json ujmJson = {{"version", _metaVersion}, {"instance_id", _replicationInstanceId}, {"auth_key", _replicationAuthKey}, {"worker", _workerId}, {"queryid", _qId}, {"uberjobid", _ujId}, - {"czarinfo", _czInfo->serializeJson()}, + {"czarinfo", _czInfo->toJson()}, {"rowlimit", _rowLimit}, - {"subqueries_map", _jobSubQueryTempMap->serializeJson()}, - {"dbtables_map", _jobDbTablesMap->serializeJson()}, + {"subqueries_map", _jobSubQueryTempMap->toJson()}, + {"dbtables_map", _jobDbTablesMap->toJson()}, {"maxtablesizemb", _maxTableSizeMB}, - {"scaninfo", _scanInfo->serializeJson()}, + {"scaninfo", _scanInfo->toJson()}, {"scaninteractive", _scanInteractive}, {"jobs", json::array()}}; auto& jsJobs = ujmJson["jobs"]; for (auto const& jbMsg : *_jobMsgVect) { - jsJobs.emplace_back(jbMsg->serializeJson()); + jsJobs.emplace_back(jbMsg->toJson()); } LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " ujmJson=" << ujmJson); @@ -138,7 +138,7 @@ UberJobMsg::Ptr UberJobMsg::createFromJson(nlohmann::json const& ujmJson) { ujmPtr->_jobSubQueryTempMap = JobSubQueryTempMap::createFromJson(jsSubQueriesMap); auto jsDbTablesMap = http::RequestBodyJSON::required(ujmJson, "dbtables_map"); - ujmPtr->_jobDbTablesMap = JobDbTablesMap::createFromJson(jsDbTablesMap); + ujmPtr->_jobDbTablesMap = JobDbTableMap::createFromJson(jsDbTablesMap); for (auto const& jsUjJob : jsUjJobs) { JobMsg::Ptr jobMsgPtr = @@ -154,13 +154,13 @@ UberJobMsg::Ptr UberJobMsg::createFromJson(nlohmann::json const& ujmJson) { JobMsg::Ptr JobMsg::create(std::shared_ptr const& jobPtr, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& jobDbTablesMap) { + JobDbTableMap::Ptr const& jobDbTablesMap) { auto jMsg = Ptr(new JobMsg(jobPtr, jobSubQueryTempMap, jobDbTablesMap)); return jMsg; } JobMsg::JobMsg(std::shared_ptr const& jobPtr, - JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap) + JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTableMap::Ptr const& jobDbTablesMap) : _jobSubQueryTempMap(jobSubQueryTempMap), _jobDbTablesMap(jobDbTablesMap) { auto const descr = jobPtr->getDescription(); if (descr == nullptr) { @@ -172,18 +172,11 @@ JobMsg::JobMsg(std::shared_ptr const& jobPtr, _chunkQuerySpecDb = chunkQuerySpec->db; _chunkId = chunkQuerySpec->chunkId; - // Add scan tables (TODO:UJ Verify this is the same for all jobs.) - for (auto const& sTbl : chunkQuerySpec->scanInfo->infoTables) { - int index = jobDbTablesMap->findDbTable(make_pair(sTbl.db, sTbl.table)); - jobDbTablesMap->setScanRating(index, sTbl.scanRating, sTbl.lockInMemory); - _chunkScanTableIndexes.push_back(index); - } - // Add fragments _jobFragments = JobFragment::createVect(*chunkQuerySpec, jobSubQueryTempMap, jobDbTablesMap); } -nlohmann::json JobMsg::serializeJson() const { +nlohmann::json JobMsg::toJson() const { auto jsJobMsg = nlohmann::json({{"jobId", _jobId}, {"attemptCount", _attemptCount}, {"querySpecDb", _chunkQuerySpecDb}, @@ -200,13 +193,13 @@ nlohmann::json JobMsg::serializeJson() const { auto& jsqFrags = jsJobMsg["queryFragments"]; for (auto& jFrag : *_jobFragments) { - jsqFrags.emplace_back(jFrag->serializeJson()); + jsqFrags.emplace_back(jFrag->toJson()); } return jsJobMsg; } -JobMsg::JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap, +JobMsg::JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTableMap::Ptr const& jobDbTablesMap, JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int chunkId) : _jobId(jobId), _attemptCount(attemptCount), @@ -217,7 +210,7 @@ JobMsg::JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap JobMsg::Ptr JobMsg::createFromJson(nlohmann::json const& ujJson, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& jobDbTablesMap) { + JobDbTableMap::Ptr const& jobDbTablesMap) { JobId jobId = http::RequestBodyJSON::required(ujJson, "jobId"); int attemptCount = http::RequestBodyJSON::required(ujJson, "attemptCount"); string chunkQuerySpecDb = http::RequestBodyJSON::required(ujJson, "querySpecDb"); @@ -231,11 +224,10 @@ JobMsg::Ptr JobMsg::createFromJson(nlohmann::json const& ujJson, jMsgPtr->_chunkScanTableIndexes = jsChunkTblIndexes.get>(); jMsgPtr->_jobFragments = JobFragment::createVectFromJson(jsQFrags, jMsgPtr->_jobSubQueryTempMap, jMsgPtr->_jobDbTablesMap); - return jMsgPtr; } -json JobSubQueryTempMap::serializeJson() const { +json JobSubQueryTempMap::toJson() const { // std::map _qTemplateMap; json jsSubQueryTemplateMap = {{"subquerytemplate_map", json::array()}}; auto& jsSqtMap = jsSubQueryTemplateMap["subquerytemplate_map"]; @@ -280,7 +272,7 @@ int JobSubQueryTempMap::findSubQueryTemp(string const& qTemp) { return index; } -int JobDbTablesMap::findDbTable(pair const& dbTablePair) { +int JobDbTableMap::findDbTable(pair const& dbTablePair) { // The expected number of templates is expected to be small, less than 4, // so this shouldn't be horribly expensive. for (auto const& [key, dbTbl] : _dbTableMap) { @@ -295,89 +287,44 @@ int JobDbTablesMap::findDbTable(pair const& dbTablePair) { return index; } -json JobDbTablesMap::serializeJson() const { - json jsDbTablesMap = {{"dbtable_map", json::array()}, {"scanrating_map", json::array()}}; - - auto& jsDbTblMap = jsDbTablesMap["dbtable_map"]; +json JobDbTableMap::toJson() const { + auto jsDbTblMap = json::array(); for (auto const& [key, valPair] : _dbTableMap) { json jsDbTbl = {{"index", key}, {"db", valPair.first}, {"table", valPair.second}}; jsDbTblMap.push_back(jsDbTbl); } - auto& jsScanRatingMap = jsDbTablesMap["scanrating_map"]; - for (auto const& [key, valPair] : _scanRatingMap) { - json jsScanR = {{"index", key}, {"scanrating", valPair.first}, {"lockinmem", valPair.second}}; - jsScanRatingMap.push_back(jsScanR); - } - - LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " " << jsDbTablesMap); - return jsDbTablesMap; + LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " " << jsDbTblMap); + return jsDbTblMap; } -JobDbTablesMap::Ptr JobDbTablesMap::createFromJson(nlohmann::json const& ujJson) { +JobDbTableMap::Ptr JobDbTableMap::createFromJson(nlohmann::json const& ujJson) { Ptr dbTablesMapPtr = create(); auto& dbTblMap = dbTablesMapPtr->_dbTableMap; - auto& scanRMap = dbTablesMapPtr->_scanRatingMap; - LOGS(_log, LOG_LVL_TRACE, "JobDbTablesMap::createFromJson " << ujJson); + LOGS(_log, LOG_LVL_TRACE, "JobDbTableMap::createFromJson " << ujJson); - json const& jsDbTbl = ujJson["dbtable_map"]; - for (auto const& jsElem : jsDbTbl) { + for (auto const& jsElem : ujJson) { int index = http::RequestBodyJSON::required(jsElem, "index"); string db = http::RequestBodyJSON::required(jsElem, "db"); string tbl = http::RequestBodyJSON::required(jsElem, "table"); auto res = dbTblMap.insert(make_pair(index, make_pair(db, tbl))); if (!res.second) { throw invalid_argument(dbTablesMapPtr->cName(__func__) + " index=" + to_string(index) + "=" + db + - +"." + tbl + " index already found in " + to_string(jsDbTbl)); - } - } - - json const& jsScanR = ujJson["scanrating_map"]; - for (auto const& jsElem : jsScanR) { - int index = http::RequestBodyJSON::required(jsElem, "index"); - int scanR = http::RequestBodyJSON::required(jsElem, "scanrating"); - bool lockInMem = http::RequestBodyJSON::required(jsElem, "lockinmem"); - auto res = scanRMap.insert(make_pair(index, make_pair(scanR, lockInMem))); - if (!res.second) { - throw invalid_argument(dbTablesMapPtr->cName(__func__) + " index=" + to_string(index) + "=" + - to_string(scanR) + +", " + to_string(lockInMem) + - " index already found in " + to_string(jsDbTbl)); + +"." + tbl + " index already found in " + to_string(ujJson)); } } return dbTablesMapPtr; } -void JobDbTablesMap::setScanRating(int index, int scanRating, bool lockInMemory) { - auto iter = _scanRatingMap.find(index); - if (iter == _scanRatingMap.end()) { - _scanRatingMap[index] = make_pair(scanRating, lockInMemory); - } else { - auto& elem = *iter; - auto& pr = elem.second; - auto& [sRating, lInMem] = pr; - if (sRating != scanRating || lInMem != lockInMemory) { - auto [dbName, tblName] = getDbTable(index); - LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " unexpected change in scanRating for " << dbName << "." << tblName - << " from " << sRating << " to " << scanRating << " lockInMemory from " - << lInMem << " to " << lockInMemory); - if (scanRating > sRating) { - sRating = scanRating; - lInMem = lockInMemory; - } - } - } -} - JobFragment::JobFragment(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& jobDbTablesMap) + JobDbTableMap::Ptr const& jobDbTablesMap) : _jobSubQueryTempMap(jobSubQueryTempMap), _jobDbTablesMap(jobDbTablesMap) {} JobFragment::VectPtr JobFragment::createVect(qproc::ChunkQuerySpec const& chunkQuerySpec, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& jobDbTablesMap) { + JobDbTableMap::Ptr const& jobDbTablesMap) { VectPtr jFragments{new Vect()}; if (chunkQuerySpec.nextFragment.get()) { qproc::ChunkQuerySpec const* sPtr = &chunkQuerySpec; @@ -401,7 +348,7 @@ JobFragment::VectPtr JobFragment::createVect(qproc::ChunkQuerySpec const& chunkQ void JobFragment::_addFragment(std::vector& jFragments, DbTableSet const& subChunkTables, std::vector const& subchunkIds, std::vector const& queries, JobSubQueryTempMap::Ptr const& subQueryTemplates, - JobDbTablesMap::Ptr const& dbTablesMap) { + JobDbTableMap::Ptr const& dbTablesMap) { LOGS(_log, LOG_LVL_TRACE, "JobFragment::_addFragment start"); Ptr jFrag = Ptr(new JobFragment(subQueryTemplates, dbTablesMap)); @@ -448,7 +395,7 @@ string JobFragment::dump() const { return os.str(); } -nlohmann::json JobFragment::serializeJson() const { +nlohmann::json JobFragment::toJson() const { json jsFragment = {{"subquerytemplate_indexes", _jobSubQueryTempIndexes}, {"dbtables_indexes", _jobDbTablesIndexes}, {"subchunkids", _subchunkIds}}; @@ -459,7 +406,7 @@ nlohmann::json JobFragment::serializeJson() const { JobFragment::VectPtr JobFragment::createVectFromJson(nlohmann::json const& jsFrags, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& dbTablesMap) { + JobDbTableMap::Ptr const& dbTablesMap) { LOGS(_log, LOG_LVL_TRACE, "JobFragment::createVectFromJson " << jsFrags); JobFragment::VectPtr jobFragments{new JobFragment::Vect()}; diff --git a/src/protojson/UberJobMsg.h b/src/protojson/UberJobMsg.h index 87c42cdba3..1449270ba2 100644 --- a/src/protojson/UberJobMsg.h +++ b/src/protojson/UberJobMsg.h @@ -65,7 +65,7 @@ class JobSubQueryTempMap { static Ptr create() { return Ptr(new JobSubQueryTempMap()); } - /// Create JobSubQueryTempMap from result of serializeJson(). + /// Create JobSubQueryTempMap from result of toJson(). static Ptr createFromJson(nlohmann::json const& ujJson); /// Find or insert qTemp into the map and return its index. @@ -75,7 +75,7 @@ class JobSubQueryTempMap { /// @throws std::out_of_range std::string getSubQueryTemp(int index) { return _qTemplateMap.at(index); } - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; private: JobSubQueryTempMap() = default; @@ -87,17 +87,17 @@ class JobSubQueryTempMap { /// The same db+table name pairs recur frequently, so the individual occurrences /// will be replaced with an integer index and use this class to recover the /// complete names. -class JobDbTablesMap { // TODO:UJ this class can probably be deleted +class JobDbTableMap { public: - using Ptr = std::shared_ptr; + using Ptr = std::shared_ptr; - std::string cName(const char* fName) const { return std::string("JobDbTablesMap::") + fName; } + std::string cName(const char* fName) const { return std::string("JobDbTableMap::") + fName; } - JobDbTablesMap(JobDbTablesMap const&) = delete; + JobDbTableMap(JobDbTableMap const&) = delete; - static Ptr create() { return Ptr(new JobDbTablesMap()); } + static Ptr create() { return Ptr(new JobDbTableMap()); } - /// Create JobDbTablesMap from result of serializeJson(). + /// Create JobDbTableMap from result of toJson(). static Ptr createFromJson(nlohmann::json const& ujJson); /// Find or insert the db.table pair into the map and return its index. @@ -107,25 +107,15 @@ class JobDbTablesMap { // TODO:UJ this class can probably be deleted /// @throws std::out_of_range std::pair getDbTable(int index) { return _dbTableMap.at(index); } - /// TODO:UJ compare with scan rating for entire UberJob - void setScanRating(int index, int scanRating, bool lockInMemory); - - /// Return scanRating(int) and lockInMemory(bool) for the dbTable at `index`. - /// TODO:UJ lockInMemory is expected to go away. - std::pair getScanRating(int index) { return _scanRatingMap[index]; } - - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; private: - JobDbTablesMap() = default; + JobDbTableMap() = default; /// Map of db name and table name pairs: db first, table second. /// The order in the map is arbitrary, but must be consistent /// so that lookups using the int index always return the same pair. std::map> _dbTableMap; - - /// Key is dbTable index, val is scanRating(int) lockInMemory(bool) - std::map> _scanRatingMap; }; /// This class stores the contents of a query fragment, which will be reconstructed @@ -143,15 +133,15 @@ class JobFragment { static VectPtr createVect(qproc::ChunkQuerySpec const& chunkQuerySpec, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& dbTablesMap); + JobDbTableMap::Ptr const& dbTablesMap); - /// Create JobFragment from the serializeJson() result. + /// Create JobFragment from the toJson() result. static VectPtr createVectFromJson(nlohmann::json const& ujJson, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& dbTablesMap); + JobDbTableMap::Ptr const& dbTablesMap); /// Return a json version of the contents of this class. - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; std::vector const& getJobSubQueryTempIndexes() const { return _jobSubQueryTempIndexes; } std::vector const& getJobDbTablesIndexes() const { return _jobDbTablesIndexes; } @@ -160,18 +150,18 @@ class JobFragment { std::string dump() const; private: - JobFragment(JobSubQueryTempMap::Ptr const& subQueryTemplates, JobDbTablesMap::Ptr const& dbTablesMap); + JobFragment(JobSubQueryTempMap::Ptr const& subQueryTemplates, JobDbTableMap::Ptr const& dbTablesMap); /// Add the required data for a query fragment. static void _addFragment(std::vector& jFragments, DbTableSet const& subChunkTables, std::vector const& subchunkIds, std::vector const& queries, JobSubQueryTempMap::Ptr const& subQueryTemplates, - JobDbTablesMap::Ptr const& dbTablesMap); + JobDbTableMap::Ptr const& dbTablesMap); JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< Pointer to indexed list of subquery fragments. std::vector _jobSubQueryTempIndexes; ///< List of subquery template indexes. - JobDbTablesMap::Ptr _jobDbTablesMap; ///< Pointer to the tables map + JobDbTableMap::Ptr _jobDbTablesMap; ///< Pointer to the tables map std::vector _jobDbTablesIndexes; ///< List of tables used. std::vector _subchunkIds; ///< List of subchunks for this chunk. @@ -192,14 +182,14 @@ class JobMsg { static Ptr create(std::shared_ptr const& jobs, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& jobDbTablesMap); + JobDbTableMap::Ptr const& jobDbTablesMap); - /// Create a Job message from the serializeJson() results. + /// Create a Job message from the toJson() results. static Ptr createFromJson(nlohmann::json const& ujJson, JobSubQueryTempMap::Ptr const& subQueryTemplates, - JobDbTablesMap::Ptr const& dbTablesMap); + JobDbTableMap::Ptr const& dbTablesMap); /// Return a json version of the contents of this class. - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; JobId getJobId() const { return _jobId; } int getAttemptCount() const { return _attemptCount; } @@ -211,19 +201,20 @@ class JobMsg { private: JobMsg(std::shared_ptr const& jobPtr, JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, - JobDbTablesMap::Ptr const& jobDbTablesMap); + JobDbTableMap::Ptr const& jobDbTablesMap); - JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTablesMap::Ptr const& jobDbTablesMap, + JobMsg(JobSubQueryTempMap::Ptr const& jobSubQueryTempMap, JobDbTableMap::Ptr const& jobDbTablesMap, JobId jobId, int attemptCount, std::string const& chunkQuerySpecDb, int chunkId); JobId _jobId; int _attemptCount; std::string _chunkQuerySpecDb; + int _chunkId; JobFragment::VectPtr _jobFragments{new JobFragment::Vect()}; JobSubQueryTempMap::Ptr _jobSubQueryTempMap; ///< Map of all query templates related to this UberJob. - JobDbTablesMap::Ptr _jobDbTablesMap; ///< Map of all db.tables related to this UberJob. + JobDbTableMap::Ptr _jobDbTablesMap; ///< Map of all db.tables related to this UberJob. // TODO:UJ remove, use value for UJ std::vector _chunkScanTableIndexes; ///< list of indexes into _jobDbTablesMap. @@ -256,7 +247,7 @@ class UberJobMsg : public std::enable_shared_from_this { static Ptr createFromJson(nlohmann::json const& ujJson); /// Return a json version of the contents of this class. - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; QueryId getQueryId() const { return _qId; } UberJobId getUberJobId() const { return _ujId; } @@ -266,7 +257,7 @@ class UberJobMsg : public std::enable_shared_from_this { CzarContactInfo::Ptr getCzarContactInfo() const { return _czInfo; } JobSubQueryTempMap::Ptr getJobSubQueryTempMap() const { return _jobSubQueryTempMap; } - JobDbTablesMap::Ptr getJobDbTablesMap() const { return _jobDbTablesMap; } + JobDbTableMap::Ptr getJobDbTableMap() const { return _jobDbTablesMap; } JobMsg::VectPtr getJobMsgVect() const { return _jobMsgVect; } @@ -298,7 +289,7 @@ class UberJobMsg : public std::enable_shared_from_this { JobSubQueryTempMap::Ptr _jobSubQueryTempMap{JobSubQueryTempMap::create()}; /// Map of all db.tables related to this UberJob. - JobDbTablesMap::Ptr _jobDbTablesMap{JobDbTablesMap::create()}; + JobDbTableMap::Ptr _jobDbTablesMap{JobDbTableMap::create()}; /// List of all job data in this UberJob. "jobs", json::array() JobMsg::VectPtr _jobMsgVect{new JobMsg::Vect()}; diff --git a/src/protojson/WorkerCzarComIssue.cc b/src/protojson/WorkerCzarComIssue.cc index c55b20b0c3..e1f3e14f54 100644 --- a/src/protojson/WorkerCzarComIssue.cc +++ b/src/protojson/WorkerCzarComIssue.cc @@ -55,9 +55,9 @@ shared_ptr WorkerCzarComIssue::toJson() const { jsCzarR["version"] = http::MetaModule::version; jsCzarR["instance_id"] = _replicationInstanceId; jsCzarR["auth_key"] = _replicationAuthKey; - jsCzarR["czarinfo"] = _czInfo->serializeJson(); + jsCzarR["czarinfo"] = _czInfo->toJson(); jsCzarR["czar"] = _czInfo->czName; - jsCzarR["workerinfo"] = _wInfo->serializeJson(); + jsCzarR["workerinfo"] = _wInfo->toJson(); jsCzarR["thoughtczarwasdead"] = _thoughtCzarWasDead; diff --git a/src/protojson/WorkerQueryStatusData.cc b/src/protojson/WorkerQueryStatusData.cc index ed4916812d..c8154b05ad 100644 --- a/src/protojson/WorkerQueryStatusData.cc +++ b/src/protojson/WorkerQueryStatusData.cc @@ -43,7 +43,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.protojson.WorkerQueryStatusData"); namespace lsst::qserv::protojson { -json CzarContactInfo::serializeJson() const { +json CzarContactInfo::toJson() const { json jsCzar; jsCzar["name"] = czName; jsCzar["id"] = czId; @@ -74,12 +74,12 @@ std::string CzarContactInfo::dump() const { return os.str(); } -json WorkerContactInfo::serializeJson() const { +json WorkerContactInfo::toJson() const { lock_guard lg(_rMtx); - return _serializeJson(); + return _toJson(); } -json WorkerContactInfo::_serializeJson() const { +json WorkerContactInfo::_toJson() const { json jsWorker; jsWorker["id"] = wId; jsWorker["host"] = _wHost; @@ -139,7 +139,7 @@ string WorkerContactInfo::_dump() const { return os.str(); } -shared_ptr WorkerQueryStatusData::serializeJson(double maxLifetime) { +shared_ptr WorkerQueryStatusData::toJson(double maxLifetime) { // Go through the _qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs lists to build a // message to send to the worker. auto now = CLOCK::now(); @@ -148,11 +148,11 @@ shared_ptr WorkerQueryStatusData::serializeJson(double maxLifetime) { jsWorkerR["version"] = http::MetaModule::version; jsWorkerR["instance_id"] = _replicationInstanceId; jsWorkerR["auth_key"] = _replicationAuthKey; - jsWorkerR["czarinfo"] = _czInfo->serializeJson(); + jsWorkerR["czarinfo"] = _czInfo->toJson(); { lock_guard lgI(_infoMtx); if (_wInfo != nullptr) { - jsWorkerR["workerinfo"] = _wInfo->serializeJson(); + jsWorkerR["workerinfo"] = _wInfo->toJson(); jsWorkerR["worker"] = _wInfo->wId; } else { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " wInfo is null"); diff --git a/src/protojson/WorkerQueryStatusData.h b/src/protojson/WorkerQueryStatusData.h index 82599bf673..5ed76de478 100644 --- a/src/protojson/WorkerQueryStatusData.h +++ b/src/protojson/WorkerQueryStatusData.h @@ -69,7 +69,7 @@ class CzarContactInfo : public std::enable_shared_from_this { uint64_t const czStartupTime; ///< czar startup time /// Return a json version of the contents of this class. - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; std::string dump() const; @@ -104,7 +104,7 @@ class WorkerContactInfo { static Ptr createFromJsonWorker(nlohmann::json const& workerJson, TIMEPOINT updateTime); /// Return a json version of the contents of this object. - nlohmann::json serializeJson() const; + nlohmann::json toJson() const; std::string cName(const char* fn) { return std::string("WorkerContactInfo::") + fn; } @@ -203,7 +203,7 @@ class WorkerContactInfo { std::string _dump() const; // _rMtx must be locked before calling - nlohmann::json _serializeJson() const; + nlohmann::json _toJson() const; std::string _wHost; ///< "host-addr" entry. std::string _wManagementHost; ///< "management-host-name" entry. @@ -293,7 +293,7 @@ class WorkerQueryStatusData { /// Create a json object held by a shared pointer to use as a message. /// Old objects in this instance will be removed after being added to the /// json message. - std::shared_ptr serializeJson(double maxLifetime); + std::shared_ptr toJson(double maxLifetime); /// Add contents of qIdDoneKeepFiles, _qIdDoneDeleteFiles, and _qIdDeadUberJobs to `jsWR`, /// and remove map elements that have an age (tmMark - element.touchTime) greater diff --git a/src/protojson/testStatusData.cc b/src/protojson/testStatusData.cc index 3135338568..b96cc63d90 100644 --- a/src/protojson/testStatusData.cc +++ b/src/protojson/testStatusData.cc @@ -57,7 +57,7 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { auto czarA = lsst::qserv::protojson::CzarContactInfo::create(czrName, czrId, czrPort, czrHost, cxrStartTime); - auto czarAJs = czarA->serializeJson(); + auto czarAJs = czarA->toJson(); auto czarB = lsst::qserv::protojson::CzarContactInfo::createFromJson(czarAJs); BOOST_REQUIRE(czarA->compare(*czarB)); @@ -72,7 +72,7 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { auto workerB = WorkerContactInfo::create("sd_workerB", "host_w2", "mgmhost_a", 3421, start); auto workerC = WorkerContactInfo::create("sd_workerC", "host_w3", "mgmhost_b", 3422, start); - auto jsWorkerA = workerA->serializeJson(); + auto jsWorkerA = workerA->toJson(); auto start1Sec = start + 1s; auto workerA1 = WorkerContactInfo::createFromJsonWorker(jsWorkerA, start1Sec); BOOST_REQUIRE(workerA->isSameContactInfo(*workerA1)); @@ -82,12 +82,12 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { replicationAuthKey); double maxLifetime = 300.0; - auto jsDataA = wqsdA->serializeJson(maxLifetime); + auto jsDataA = wqsdA->toJson(maxLifetime); // Check that empty lists work. auto wqsdA1 = lsst::qserv::protojson::WorkerQueryStatusData::createFromJson( *jsDataA, replicationInstanceId, replicationAuthKey, start1Sec); - auto jsDataA1 = wqsdA1->serializeJson(maxLifetime); + auto jsDataA1 = wqsdA1->toJson(maxLifetime); BOOST_REQUIRE(*jsDataA == *jsDataA1); vector qIdsDelFiles = {7, 8, 9, 15, 25, 26, 27, 30}; @@ -96,7 +96,7 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { wqsdA->qIdDoneDeleteFiles[qIdDF] = start; } - jsDataA = wqsdA->serializeJson(maxLifetime); + jsDataA = wqsdA->toJson(maxLifetime); BOOST_REQUIRE(*jsDataA != *jsDataA1); for (auto const qIdKF : qIdsKeepFiles) { @@ -105,24 +105,24 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { wqsdA->addDeadUberJobs(12, {1, 3}, start); - jsDataA = wqsdA->serializeJson(maxLifetime); + jsDataA = wqsdA->toJson(maxLifetime); auto start5Sec = start + 5s; auto workerAFromJson = lsst::qserv::protojson::WorkerQueryStatusData::createFromJson( *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); - auto jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); + auto jsWorkerAFromJson = workerAFromJson->toJson(maxLifetime); BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); wqsdA->addDeadUberJobs(12, {34}, start5Sec); wqsdA->addDeadUberJobs(91, {77}, start5Sec); wqsdA->addDeadUberJobs(1059, {1, 4, 6, 7, 8, 10, 3, 22, 93}, start5Sec); - jsDataA = wqsdA->serializeJson(maxLifetime); + jsDataA = wqsdA->toJson(maxLifetime); BOOST_REQUIRE(*jsDataA != *jsWorkerAFromJson); workerAFromJson = lsst::qserv::protojson::WorkerQueryStatusData::createFromJson( *jsDataA, replicationInstanceId, replicationAuthKey, start5Sec); - jsWorkerAFromJson = workerAFromJson->serializeJson(maxLifetime); + jsWorkerAFromJson = workerAFromJson->toJson(maxLifetime); BOOST_REQUIRE(*jsDataA == *jsWorkerAFromJson); // Make the response, which contains lists of the items handled by the workers. @@ -155,11 +155,11 @@ BOOST_AUTO_TEST_CASE(WorkerCzarComIssue) { auto czarA = lsst::qserv::protojson::CzarContactInfo::create(czrName, czrId, czrPort, czrHost, cxrStartTime); - auto czarAJs = czarA->serializeJson(); + auto czarAJs = czarA->toJson(); auto start = lsst::qserv::CLOCK::now(); auto workerA = WorkerContactInfo::create("sd_workerA", "host_w1", "mgmhost_a", 3421, start); - auto jsWorkerA = workerA->serializeJson(); + auto jsWorkerA = workerA->toJson(); // WorkerCzarComIssue auto wccIssueA = diff --git a/src/protojson/testUberJobMsg.cc b/src/protojson/testUberJobMsg.cc index a16a047b7c..14cb74e7fe 100644 --- a/src/protojson/testUberJobMsg.cc +++ b/src/protojson/testUberJobMsg.cc @@ -49,13 +49,13 @@ BOOST_AUTO_TEST_SUITE(Suite) string testA() { string ta = - R"({"maxtablesizemb":5432,"scaninteractive":true,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":{"dbtable_map":[],"scanrating_map":[]},"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":51,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; + R"({"maxtablesizemb":5432,"scaninteractive":true,"auth_key":"replauthkey","czarinfo":{"czar-startup-time":1732658208085,"id":1,"management-host-name":"3a8b68cf9b67","management-port":40865,"name":"proxy"},"dbtables_map":[],"scaninfo":{"infoscanrating":0,"infotables":[]},"instance_id":"qserv_proj","jobs":[{"attemptCount":0,"chunkId":1234567890,"chunkresultname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","chunkscantables_indexes":[],"jobId":0,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_1_a0d45001254932466b784acf90323565_1234567890_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"qcase01","scanInteractive":true,"scanPriority":0}],"queryid":1,"rowlimit":0,"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT `qcase01.Filter`.`filterId` AS `filterId`,`qcase01.Filter`.`filterName` AS `filterName`,`qcase01.Filter`.`photClam` AS `photClam`,`qcase01.Filter`.`photBW` AS `photBW` FROM `qcase01`.`Filter`AS`qcase01.Filter` WHERE (`qcase01.Filter`.`filterId`<<1)=2"}]},"uberjobid":2,"version":51,"worker":"6c56ba9b-ac40-11ef-acb7-0242c0a8030a"})"; return ta; } string testB() { string tb = - R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":{"dbtable_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"scanrating_map":[{"index":0,"lockinmem":true,"scanrating":1}]},"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"scaninteractive":false,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":51,"worker":"db04"})"; + R"({"auth_key":"slac6dev:kukara4a","czarinfo":{"czar-startup-time":1733499789161,"id":7,"management-host-name":"sdfqserv001.sdf.slac.stanford.edu","management-port":41923,"name":"proxy"},"dbtables_map":[{"db":"dp02_dc2_catalogs","index":0,"table":"Object"}],"instance_id":"slac6dev","jobs":[{"attemptCount":0,"chunkId":79680,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","chunkscantables_indexes":[0],"jobId":1398,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_79680_0","subchunkids":[],"subquerytemplate_indexes":[0]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":80358,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","chunkscantables_indexes":[0],"jobId":1435,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_80358_0","subchunkids":[],"subquerytemplate_indexes":[1]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1},{"attemptCount":0,"chunkId":81017,"chunkresultname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","chunkscantables_indexes":[0],"jobId":1452,"queryFragments":[{"dbtables_indexes":[],"resulttblname":"r_280607_e6eac6bb53b0f8505ed36bf82a4d93f1_81017_0","subchunkids":[],"subquerytemplate_indexes":[2]}],"querySpecDb":"dp02_dc2_catalogs","scanInteractive":false,"scanPriority":1}],"maxtablesizemb":5100,"scaninteractive":false,"queryid":280607,"rowlimit":0,"scaninfo":{"infoscanrating":1,"infotables":[{"sidb":"dp02_dc2_catalogs","silockinmem":true,"sirating":1,"sitable":"Object"}]},"subqueries_map":{"subquerytemplate_map":[{"index":0,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_79680` AS `obj`"},{"index":1,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_80358` AS `obj`"},{"index":2,"template":"SELECT COUNT(`obj`.`g_ap12Flux`) AS `QS1_COUNT`,SUM(`obj`.`g_ap12Flux`) AS `QS2_SUM`,MIN(`obj`.`g_ap12Flux`) AS `QS3_MIN`,MAX(`obj`.`g_ap12Flux`) AS `QS4_MAX`,COUNT(`obj`.`g_ap12FluxErr`) AS `QS5_COUNT`,SUM(`obj`.`g_ap12FluxErr`) AS `QS6_SUM`,MIN(`obj`.`g_ap12FluxErr`) AS `QS7_MIN`,MAX(`obj`.`g_ap12FluxErr`) AS `QS8_MAX`,COUNT(`obj`.`g_ap25Flux`) AS `QS9_COUNT`,SUM(`obj`.`g_ap25Flux`) AS `QS10_SUM`,MIN(`obj`.`g_ap25Flux`) AS `QS11_MIN`,MAX(`obj`.`g_ap25Flux`) AS `QS12_MAX`,COUNT(`obj`.`g_ap25FluxErr`) AS `QS13_COUNT`,SUM(`obj`.`g_ap25FluxErr`) AS `QS14_SUM`,MIN(`obj`.`g_ap25FluxErr`) AS `QS15_MIN`,MAX(`obj`.`g_ap25FluxErr`) AS `QS16_MAX` FROM `dp02_dc2_catalogs`.`Object_81017` AS `obj`"}]},"uberjobid":147,"version":51,"worker":"db04"})"; return tb; } @@ -69,12 +69,12 @@ bool parseSerializeReparseCheck(string const& jsStr, string const& note) { UberJobMsg::Ptr ujm = UberJobMsg::createFromJson(js); BOOST_REQUIRE(ujm != nullptr); - nlohmann::json jsUjm = ujm->serializeJson(); + nlohmann::json jsUjm = ujm->toJson(); LOGS(_log, LOG_LVL_INFO, fName << " serialized jsUjm=" << jsUjm); UberJobMsg::Ptr ujmCreated = UberJobMsg::createFromJson(jsUjm); LOGS(_log, LOG_LVL_INFO, fName << " created"); - nlohmann::json jsUjmCreated = ujmCreated->serializeJson(); + nlohmann::json jsUjmCreated = ujmCreated->toJson(); LOGS(_log, LOG_LVL_INFO, fName << " created->serialized"); bool createdMatchesOriginal = jsUjm == jsUjmCreated; @@ -99,14 +99,14 @@ BOOST_AUTO_TEST_CASE(WorkerQueryStatusData) { UberJobMsg::Ptr ujm = UberJobMsg::createFromJson(js); BOOST_REQUIRE(ujm != nullptr); - nlohmann::json jsUjm = ujm->serializeJson(); + nlohmann::json jsUjm = ujm->toJson(); LOGS(_log, LOG_LVL_INFO, "js=" << js); LOGS(_log, LOG_LVL_INFO, "jsUjm=" << jsUjm); UberJobMsg::Ptr ujmCreated = UberJobMsg::createFromJson(jsUjm); LOGS(_log, LOG_LVL_INFO, "ujmCreated=" << ujmCreated); - nlohmann::json jsUjmCreated = ujmCreated->serializeJson(); + nlohmann::json jsUjmCreated = ujmCreated->toJson(); bool createdMatchesOriginal = jsUjm == jsUjmCreated; if (!createdMatchesOriginal) { diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 076df78e9e..7e69c96d32 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -134,7 +134,6 @@ Executive::Ptr Executive::create(int secsBetweenUpdates, shared_ptrserializeJson(); + json request = uberJobMsg->toJson(); jobsLock.unlock(); // unlock so other _jobsMtx threads can advance while this waits for transmit diff --git a/src/util/Command.cc b/src/util/Command.cc index 39cb7d4d84..7ce436361b 100644 --- a/src/util/Command.cc +++ b/src/util/Command.cc @@ -26,6 +26,9 @@ // Class header #include "util/Command.h" +// System headers +#include + // LSST headers #include "lsst/log/Log.h" @@ -33,6 +36,8 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.util.Command"); } +using namespace std; + namespace lsst::qserv::util { /// Set status to COMPLETE and notify everyone waiting for a status change. @@ -70,4 +75,20 @@ void Command::setFunc(std::function func) { /// this function must be called or the lambda will keep this object alive. void Command::resetFunc() { setFunc(nullptr); } +std::string Command::dump() const { + ostringstream os; + dump(os); + return os.str(); +} + +std::ostream& Command::dump(std::ostream& os) const { + os << "util::Command"; + return os; +} + +ostream& operator<<(ostream& os, Command const& cmd) { + cmd.dump(os); + return os; +} + } // namespace lsst::qserv::util diff --git a/src/util/Command.h b/src/util/Command.h index 6d28f87271..9456d1981e 100644 --- a/src/util/Command.h +++ b/src/util/Command.h @@ -79,6 +79,11 @@ class Command { void setFunc(std::function func); void resetFunc(); + /// Returns a string for logging. + virtual std::ostream& dump(std::ostream& os) const; + std::string dump() const; + friend std::ostream& operator<<(std::ostream& os, Command const& cmd); + protected: std::function _func = [](CmdData*) { ; }; }; diff --git a/src/util/InstanceCount.cc b/src/util/InstanceCount.cc index 69ebc4485b..80b920b1e4 100644 --- a/src/util/InstanceCount.cc +++ b/src/util/InstanceCount.cc @@ -9,8 +9,6 @@ // LSST headers #include "lsst/log/Log.h" -// qserv headers - using namespace std; namespace { // File-scope helpers diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index 54a61ba07a..b80689fa7d 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -445,6 +445,7 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptrgetIdStr() << " sending done!!!"); } + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared " << task->cName(__func__) << " sending done!!!"); } } transmitT.stop(); diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 24e4e65908..63a14311f4 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -202,7 +202,7 @@ std::vector Task::createTasksFromUberJobMsg( std::string workerId = ujMsg->getWorkerId(); auto jobSubQueryTempMap = ujMsg->getJobSubQueryTempMap(); - auto jobDbTablesMap = ujMsg->getJobDbTablesMap(); + auto jobDbTablesMap = ujMsg->getJobDbTableMap(); auto jobMsgVect = ujMsg->getJobMsgVect(); for (auto const& jobMsg : *jobMsgVect) { @@ -272,81 +272,72 @@ std::vector Task::createTasksFromUberJobMsg( std::vector Task::createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, std::shared_ptr const& sendChannel, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr) { + std::shared_ptr const& chunkResourceMgr, + std::shared_ptr const& queriesAndChunks) { + vector vect; QueryId qId = ujData->getQueryId(); - UberJobId ujId = ujData->getUberJobId(); CzarIdType czId = ujData->getCzarId(); string funcN(__func__); funcN += " QID=" + to_string(qId) + " czId=" + to_string(czId); + protojson::JobSubQueryTempMap::Ptr jobSubQueryTempMap{protojson::JobSubQueryTempMap::create()}; + protojson::JobDbTableMap::Ptr jobDbTablesMap{protojson::JobDbTableMap::create()}; + protojson::JobMsg::VectPtr jobMsgVect{new protojson::JobMsg::Vect()}; + for (auto const& jsUjJob : jsJobs) { + protojson::JobMsg::Ptr jobMsgPtr = + protojson::JobMsg::createFromJson(jsUjJob, jobSubQueryTempMap, jobDbTablesMap); + jobMsgVect->push_back(jobMsgPtr); + } - vector vect; - for (auto const& job : jsJobs) { - json const& jsJobDesc = job["jobdesc"]; - http::RequestBodyJSON rbJobDesc(jsJobDesc); - // See qproc::TaskMsgFactory::makeMsgJson for message construction. - auto const jdCzarId = rbJobDesc.required("czarId"); - auto const jdQueryId = rbJobDesc.required("queryId"); - if (jdQueryId != qId) { - throw TaskException(ERR_LOC, string("ujId=") + to_string(ujId) + " qId=" + to_string(qId) + - " QueryId mismatch Job qId=" + to_string(jdQueryId)); - } - auto const jdJobId = rbJobDesc.required("jobId"); - auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); - auto const jdChunkId = rbJobDesc.required("chunkId"); - LOGS(_log, LOG_LVL_TRACE, - funcN << " jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId - << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); + wpublish::QueryStatistics::Ptr queryStats = queriesAndChunks->addQueryId(qId, czId); + UserQueryInfo::Ptr userQueryInfo = queryStats->getUserQueryInfo(); + + for (auto const& jobMsg : *jobMsgVect) { + JobId jobId = jobMsg->getJobId(); + int attemptCount = jobMsg->getAttemptCount(); + std::string chunkQuerySpecDb = jobMsg->getChunkQuerySpecDb(); + int chunkId = jobMsg->getChunkId(); - auto const jdQueryFragments = rbJobDesc.required("queryFragments"); + std::vector chunkScanTableIndexes = jobMsg->getChunkScanTableIndexes(); + auto jobFragments = jobMsg->getJobFragments(); int fragmentNumber = 0; - for (auto const& frag : jdQueryFragments) { + + for (auto const& fMsg : *jobFragments) { + // These need to be constructed for the fragment vector fragSubQueries; - vector fragSubchunkIds; vector fragSubTables; - LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); - http::RequestBodyJSON rbFrag(frag); - auto const& jsQueries = rbFrag.required("queries"); - // TODO:UJ move to uberjob???, these should be the same for all jobs - for (auto const& subQ : jsQueries) { - http::RequestBodyJSON rbSubQ(subQ); - auto const subQuery = rbSubQ.required("subQuery"); - LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); - fragSubQueries.push_back(subQuery); - } - auto const& resultTable = rbFrag.required("resultTable"); - auto const& jsSubIds = rbFrag.required("subchunkIds"); - for (auto const& scId : jsSubIds) { - fragSubchunkIds.push_back(scId); + vector fragSubchunkIds; + + vector fsqIndexes = fMsg->getJobSubQueryTempIndexes(); + for (int fsqIndex : fsqIndexes) { + string fsqStr = jobSubQueryTempMap->getSubQueryTemp(fsqIndex); + fragSubQueries.push_back(fsqStr); } - auto const& jsSubTables = rbFrag.required("subchunkTables"); - for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? - http::RequestBodyJSON rbScDbTable(scDbTable); - string scDb = rbScDbTable.required("scDb"); - string scTable = rbScDbTable.required("scTable"); + vector dbTblIndexes = fMsg->getJobDbTablesIndexes(); + for (int dbTblIndex : dbTblIndexes) { + auto [scDb, scTable] = jobDbTablesMap->getDbTable(dbTblIndex); TaskDbTbl scDbTbl(scDb, scTable); fragSubTables.push_back(scDbTbl); } + fragSubchunkIds = fMsg->getSubchunkIds(); + for (string const& fragSubQ : fragSubQueries) { - LOGS(_log, LOG_LVL_DEBUG, "ignoring " << fragSubQ); + size_t templateId = userQueryInfo->addTemplate(fragSubQ); if (fragSubchunkIds.empty()) { bool const noSubchunks = false; int const subchunkId = -1; - auto task = Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, - 0, noSubchunks, subchunkId, jdQuerySpecDb, fragSubTables, - fragSubchunkIds, sendChannel, nullptr)); + auto task = Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, + templateId, noSubchunks, subchunkId, chunkQuerySpecDb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats)); vect.push_back(task); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; auto task = - Task::Ptr(new Task(ujData, jdJobId, jdAttemptCount, jdChunkId, fragmentNumber, - 0, hasSubchunks, subchunkId, jdQuerySpecDb, fragSubTables, - fragSubchunkIds, sendChannel, nullptr)); + Task::Ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, + templateId, hasSubchunks, subchunkId, chunkQuerySpecDb, + fragSubTables, fragSubchunkIds, sendChannel, queryStats)); vect.push_back(task); } } @@ -588,10 +579,19 @@ nlohmann::json Task::getJson() const { int64_t Task::getMaxTableSize() const { return _ujData->getMaxTableSizeBytes(); } -ostream& operator<<(ostream& os, Task const& t) { +ostream& Task::dump(ostream& os) const { os << "Task: " - << "msg: " << t.getIdStr() << " chunk=" << t._chunkId << " db=" << t._db << " " << t.getQueryString(); - + << "msg: " << getIdStr() << " chunk=" << _chunkId << " seq=" << _tSeq << " db=" << _db << " " + << getQueryString(); + if (_ujData == nullptr) { + os << " ujData=null"; + return os; + } + if (_ujData->getScanInfo() == nullptr) { + os << " scanInfo=null"; + return os; + } + _ujData->getScanInfo()->dump(os); return os; } diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 66eabf8ed2..b1da3c6ae9 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -172,7 +172,8 @@ class Task : public util::CommandForThreadPool { static std::vector createTasksForUnitTest( std::shared_ptr const& ujData, nlohmann::json const& jsJobs, std::shared_ptr const& sendChannel, int maxTableSizeMb, - std::shared_ptr const& chunkResourceMgr); + std::shared_ptr const& chunkResourceMgr, + std::shared_ptr const& queriesAndChunks); std::shared_ptr getSendChannel() const { return _sendChannel; } std::string user; ///< Incoming username @@ -211,7 +212,6 @@ class Task : public util::CommandForThreadPool { void freeTaskQueryRunner(wdb::QueryRunner* tqr); void setTaskScheduler(TaskScheduler::Ptr const& scheduler) { _taskScheduler = scheduler; } TaskScheduler::Ptr getTaskScheduler() const { return _taskScheduler.lock(); } - friend std::ostream& operator<<(std::ostream& os, Task const& t); // Shared scan information bool getHasChunkId() const { return _hasChunkId; } @@ -309,6 +309,8 @@ class Task : public util::CommandForThreadPool { int getLvlWT() const { return _logLvlWT; } int getLvlET() const { return _logLvlET; } + std::ostream& dump(std::ostream& os) const override; + private: std::atomic _logLvlWT; ///< Normally LOG_LVL_WARN, set to TRACE in cancelled Tasks. std::atomic _logLvlET; ///< Normally LOG_LVL_ERROR, set to TRACE in cancelled Tasks. diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index e4a59e3d20..55ca778613 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -102,6 +102,13 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount cName(__func__) << " _responseState was " << _responseState << " instead of NOTHING"); } + // Latch to prevent errors from being transmitted. + // NOTE: Calls to responseError() and responseFileReady() are protected by the + // mutex in FileChannelShared (_tMtx). + if (_responseState.exchange(SENDING_FILEURL) != NOTHING) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << " _responseState was " << _responseState << " instead of NOTHING"); + } string workerIdStr; if (_foreman != nullptr) { workerIdStr = _foreman->chunkInventory()->id(); diff --git a/src/wcomms/HttpWorkerCzarModule.h b/src/wcomms/HttpWorkerCzarModule.h index 5ea2b956a3..8894b72a6e 100644 --- a/src/wcomms/HttpWorkerCzarModule.h +++ b/src/wcomms/HttpWorkerCzarModule.h @@ -39,6 +39,11 @@ class CzarContactInfo; class UberJobMsg; } // namespace lsst::qserv::protojson +namespace lsst::qserv::protojson { +class CzarContactInfo; +class UberJobMsg; +} // namespace lsst::qserv::protojson + namespace lsst::qserv::qhttp { class Request; class Response; diff --git a/src/wdb/CMakeLists.txt b/src/wdb/CMakeLists.txt index 7b4159ec35..8137ee3869 100644 --- a/src/wdb/CMakeLists.txt +++ b/src/wdb/CMakeLists.txt @@ -45,9 +45,5 @@ ENDFUNCTION() wdb_tests( testChunkResource - testQueryRunner ) -# For this test to work, a mariadb server needs to be available. -# This functionality is covered by integration tests. -set_tests_properties(testQueryRunner PROPERTIES WILL_FAIL 1) diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index e5f0e6c92a..53f581bb51 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -151,7 +151,6 @@ bool QueryRunner::runQuery() { return false; } - LOGS(_log, LOG_LVL_TRACE, "QR in flight for sqlConnMgr " << _sqlConnMgr->dump()); // Queries that span multiple tasks should not be high priority for the SqlConMgr as it risks deadlock. bool interactive = _task->getScanInteractive() && !(_task->getSendChannel()->getTaskCount() > 1); wcontrol::SqlConnLock sqlConnLock(*_sqlConnMgr, not interactive, _task->getSendChannel()); diff --git a/src/wdb/testQueryRunner.cc b/src/wdb/testQueryRunner.cc deleted file mode 100644 index 17731d05f9..0000000000 --- a/src/wdb/testQueryRunner.cc +++ /dev/null @@ -1,256 +0,0 @@ -// -*- LSST-C++ -*- -/* - * LSST Data Management System - * Copyright 2014-2016 AURA/LSST. - * - * This product includes software developed by the - * LSST Project (http://www.lsst.org/). - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the LSST License Statement and - * the GNU General Public License along with this program. If not, - * see . - */ -/** - * @brief Simple testing for class QueryRunner - * Requires some setup, and assumes some access to a mysqld - * - * @author Daniel L. Wang, SLAC - */ - -// Qserv headers -#include "mysql/MySqlConfig.h" -#include "protojson/ScanTableInfo.h" -#include "wbase/FileChannelShared.h" -#include "wbase/Task.h" -#include "wbase/UberJobData.h" -#include "wconfig/WorkerConfig.h" -#include "wcontrol/Foreman.h" -#include "wcontrol/SqlConnMgr.h" -#include "wdb/ChunkResource.h" -#include "wdb/QueryRunner.h" -#include "wpublish/QueriesAndChunks.h" - -// Boost unit test header -#define BOOST_TEST_MODULE QueryRunner -#include - -using namespace std; - -namespace test = boost::test_tools; -namespace util = lsst::qserv::util; - -using lsst::qserv::mysql::MySqlConfig; -using lsst::qserv::mysql::MySqlConnection; - -using lsst::qserv::wbase::FileChannelShared; -using lsst::qserv::wbase::SendChannel; -using lsst::qserv::wbase::Task; -using lsst::qserv::wconfig::WorkerConfig; -using lsst::qserv::wcontrol::SqlConnMgr; -using lsst::qserv::wdb::ChunkResource; -using lsst::qserv::wdb::ChunkResourceMgr; -using lsst::qserv::wdb::FakeBackend; -using lsst::qserv::wdb::QueryRunner; -using lsst::qserv::wpublish::QueriesAndChunks; - -struct Fixture { - struct MsgInfo { - string const db = "LSST"; - string const table = "Object"; - string const qry = "SELECT AVG(yFlux_PS) from LSST.Object_3240"; - int const chunkId = 3240; - int const czarId = 5; - string const czarName = "cz5"; - string const czarHostName = "cz5host"; - int const czarPort = 3437; - string const targWorkerId = "a_worker"; - std::shared_ptr foreman; - int const queryId = 23; - int const jobId = 1; - int const uberJobId = 1; - int const attemptCount = 1; - int const scanRating = 1; - bool const scanInteractive = false; - int const maxTableSize = 5000; - bool const lockInMemory = false; - string const resultName = "resName"; - string const authKey = "noAuthKey"; - int const rowLimit = 0; - }; - - shared_ptr newTaskJson(MsgInfo const& mInfo) { - // Derived from TaskMsgFactory::makeMsgJson - - auto jsJobMsgPtr = std::shared_ptr( - new nlohmann::json({{"czarId", mInfo.czarId}, - {"queryId", mInfo.queryId}, - {"jobId", mInfo.jobId}, - {"attemptCount", mInfo.attemptCount}, - {"querySpecDb", mInfo.db}, - {"maxTableSize", mInfo.maxTableSize}, - {"chunkScanTables", nlohmann::json::array()}, - {"chunkId", mInfo.chunkId}, - {"queryFragments", nlohmann::json::array()}})); - - auto& jsJobMsg = *jsJobMsgPtr; - - auto& chunkScanTables = jsJobMsg["chunkScanTables"]; - nlohmann::json cst = {{"db", mInfo.db}, - {"table", mInfo.table}, - {"lockInMemory", mInfo.lockInMemory}, - {"tblScanRating", mInfo.scanRating}}; - chunkScanTables.push_back(move(cst)); - - auto& jsFragments = jsJobMsg["queryFragments"]; - nlohmann::json jsFrag = {{"resultTable", mInfo.resultName}, - {"queries", nlohmann::json::array()}, - {"subchunkTables", nlohmann::json::array()}, - {"subchunkIds", nlohmann::json::array()}}; - - auto& jsQueries = jsFrag["queries"]; - nlohmann::json jsQry = {{"subQuery", mInfo.qry}}; - jsQueries.push_back(move(jsQry)); - - jsFragments.push_back(move(jsFrag)); - - return jsJobMsgPtr; - } - - MySqlConfig newMySqlConfig() { - string user = "qsmaster"; - string password = ""; - string socket = "SET ME HERE"; - MySqlConfig mySqlConfig(user, password, socket); - if (not MySqlConnection::checkConnection(mySqlConfig)) { - throw runtime_error("Unable to connect to MySQL database with params: " + mySqlConfig.toString()); - } - return mySqlConfig; - } - shared_ptr queriesAndChunks() { - bool resetForTesting = true; - int maxTasksBooted = 5; - int maxDarkTasks = 25; - return QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(120), maxTasksBooted, - maxDarkTasks, resetForTesting); - } -}; - -BOOST_FIXTURE_TEST_SUITE(Basic, Fixture, *boost::unit_test::timeout(20)) - -BOOST_AUTO_TEST_CASE(Simple) { - /* &&& - WorkerConfig::create(); - MsgInfo mInfo; - auto msgJson = newTaskJson(mInfo); - shared_ptr sendC(SendChannel::newNopChannel()); - auto sChannel = FileChannelShared::create(sendC, mInfo.czarId); - FakeBackend::Ptr backend = make_shared(); - shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); - auto const queries = queriesAndChunks(); - - auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); - scanInfo->scanRating = mInfo.scanRating; - scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, - mInfo.foreman, mInfo.authKey); - - vector taskVect = - Task::createTasksForUnitTest(ujData, *msgJson, sChannel, mInfo.maxTableSize, crm); - - Task::Ptr task = taskVect[0]; - QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); - BOOST_CHECK(a->runQuery()); - */ - - WorkerConfig::create(); - MsgInfo mInfo; - auto msgJson = newTaskJson(mInfo); - shared_ptr sendC(SendChannel::newNopChannel()); - - auto sChannel = FileChannelShared::create(sendC, mInfo.czarId); - FakeBackend::Ptr backend = make_shared(); - shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); - auto const queries = queriesAndChunks(); - - auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); - scanInfo->scanRating = mInfo.scanRating; - scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, - mInfo.foreman, mInfo.authKey); - - vector taskVect = - Task::createTasksForUnitTest(ujData, *msgJson, sChannel, mInfo.maxTableSize, crm); - - Task::Ptr task = taskVect[0]; - QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); - BOOST_CHECK(a->runQuery()); -} - -BOOST_AUTO_TEST_CASE(Output) { - /* &&& - WorkerConfig::create(); - string out; - MsgInfo mInfo; - auto msgJson = newTaskJson(mInfo); - shared_ptr sendC(SendChannel::newStringChannel(out)); - auto sc = FileChannelShared::create(sendC, mInfo.czarId); - FakeBackend::Ptr backend = make_shared(); - shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); - auto const queries = queriesAndChunks(); - auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); - scanInfo->scanRating = mInfo.scanRating; - scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, - mInfo.foreman, mInfo.authKey); - - vector taskVect = Task::createTasksForUnitTest(ujData, *msgJson, sc, mInfo.maxTableSize, crm); - - Task::Ptr task = taskVect[0]; - QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); - BOOST_CHECK(a->runQuery()); - */ - WorkerConfig::create(); - string out; - MsgInfo mInfo; - auto msgJson = newTaskJson(mInfo); - shared_ptr sendC(SendChannel::newStringChannel(out)); - auto sc = FileChannelShared::create(sendC, mInfo.czarId); - FakeBackend::Ptr backend = make_shared(); - shared_ptr crm = ChunkResourceMgr::newMgr(backend); - SqlConnMgr::Ptr sqlConnMgr = make_shared(20, 9); - auto const queries = queriesAndChunks(); - auto scanInfo = lsst::qserv::protojson::ScanInfo::create(); - scanInfo->scanRating = mInfo.scanRating; - scanInfo->infoTables.emplace_back(mInfo.db, mInfo.table, mInfo.lockInMemory, mInfo.scanRating); - auto ujData = lsst::qserv::wbase::UberJobData::create( - mInfo.uberJobId, mInfo.czarName, mInfo.czarId, mInfo.czarHostName, mInfo.czarPort, mInfo.queryId, - mInfo.rowLimit, mInfo.maxTableSize, scanInfo, mInfo.scanInteractive, mInfo.targWorkerId, - mInfo.foreman, mInfo.authKey); - - vector taskVect = Task::createTasksForUnitTest(ujData, *msgJson, sc, mInfo.maxTableSize, crm); - - Task::Ptr task = taskVect[0]; - QueryRunner::Ptr a(QueryRunner::newQueryRunner(task, crm, newMySqlConfig(), sqlConnMgr, queries)); - BOOST_CHECK(a->runQuery()); -} - -BOOST_AUTO_TEST_SUITE_END() diff --git a/src/wmain/WorkerMain.cc b/src/wmain/WorkerMain.cc index bcab50f297..d26aeb3fd3 100644 --- a/src/wmain/WorkerMain.cc +++ b/src/wmain/WorkerMain.cc @@ -238,7 +238,6 @@ void WorkerMain::terminate() { { lock_guard lck(_terminateMtx); if (_terminate.exchange(true)) return; - ; } _terminateCv.notify_all(); _controlHttpSvc->stop(); diff --git a/src/wpublish/QueriesAndChunks.cc b/src/wpublish/QueriesAndChunks.cc index 1410e819d2..ec4b2e0053 100644 --- a/src/wpublish/QueriesAndChunks.cc +++ b/src/wpublish/QueriesAndChunks.cc @@ -93,11 +93,21 @@ QueriesAndChunks::QueriesAndChunks(chrono::seconds deadAfter, chrono::seconds ex } auto rExamine = [this]() { + int examineAfterSeconds = _examineAfter.count(); + int seconds = 0; while (_loopExamine) { - this_thread::sleep_for(_examineAfter); - if (_loopExamine) examineAll(); + // Check frequently so unit tests finish in less time, + // as they aren't waiting for a 5 minute sleep to finish. + this_thread::sleep_for(chrono::seconds(2)); + seconds++; + if (_loopExamine && seconds > examineAfterSeconds) { + examineAll(); + seconds = 0; + examineAfterSeconds = _examineAfter.count(); + } } }; + thread te(rExamine); _examineThread = move(te); } diff --git a/src/wsched/BlendScheduler.cc b/src/wsched/BlendScheduler.cc index 9fd8a883b3..06c90af3ae 100644 --- a/src/wsched/BlendScheduler.cc +++ b/src/wsched/BlendScheduler.cc @@ -420,6 +420,11 @@ int BlendScheduler::_getAdjustedMaxThreads(int oldAdjMax, int inFlight) { return newAdjMax; } +int BlendScheduler::calcAvailableTheads() { + lock_guard lck(_schedMtx); + return _calcAvailableTheads(); +} + /// @return the number of threads that are not reserved by any sub-scheduler. int BlendScheduler::_calcAvailableTheads() { int reserve = 0; diff --git a/src/wsched/BlendScheduler.h b/src/wsched/BlendScheduler.h index e25eba7bd6..5507c755fd 100644 --- a/src/wsched/BlendScheduler.h +++ b/src/wsched/BlendScheduler.h @@ -132,6 +132,8 @@ class BlendScheduler : public wsched::SchedulerBase { /// Do nothing, the schedulers this class manages keep their own statistics. void recordPerformanceData() override {}; + int calcAvailableTheads(); + private: int _getAdjustedMaxThreads(int oldAdjMax, int inFlight); bool _ready(); diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index cafbfb8dc2..bf1d4a96f7 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -161,6 +161,7 @@ bool ChunkTasksQueue::_ready(bool useFlexibleLock) { LOGS(_log, LOG_LVL_TRACE, "_ready loopB state=" << ChunkTasks::toStr(chunkState) << " iter=" << iter->first << " " << iter->second->cInfo()); + _readyChunk = iter->second; return true; } diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index fb87bf3db8..b1156b10ef 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -115,6 +115,11 @@ bool ScanScheduler::ready() { return _ready(); } +string ScanScheduler::getRatingStr() const { + string const str = "min=" + to_string(_minRating) + " max=" + to_string(_maxRating); + return str; +} + /// Precondition: _mx is locked /// Returns true if there is a Task ready to go and we aren't up against any limits. bool ScanScheduler::_ready() { diff --git a/src/wsched/ScanScheduler.h b/src/wsched/ScanScheduler.h index e1739c7afe..7f65276def 100644 --- a/src/wsched/ScanScheduler.h +++ b/src/wsched/ScanScheduler.h @@ -68,7 +68,8 @@ class ScanScheduler : public SchedulerBase { util::Command::Ptr getCmd(bool wait) override; void commandStart(util::Command::Ptr const& cmd) override; void commandFinish(util::Command::Ptr const& cmd) override; - bool isRatingInRange(int rating) { return _minRating <= rating && rating <= _maxRating; } + bool isRatingInRange(int rating) const { return _minRating <= rating && rating <= _maxRating; } + std::string getRatingStr() const; // SchedulerBase overrides bool ready() override; diff --git a/src/wsched/testSchedulers.cc b/src/wsched/testSchedulers.cc index 7375991814..8b4126f755 100644 --- a/src/wsched/testSchedulers.cc +++ b/src/wsched/testSchedulers.cc @@ -37,6 +37,7 @@ #include "util/EventThread.h" #include "wbase/FileChannelShared.h" #include "wbase/Task.h" +#include "wbase/UberJobData.h" #include "wconfig/WorkerConfig.h" #include "wcontrol/SqlConnMgr.h" #include "wpublish/QueriesAndChunks.h" @@ -59,9 +60,14 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wsched.testSchedulers"); using namespace std; using lsst::qserv::mysql::MySqlConfig; +using lsst::qserv::protojson::ScanInfo::Rating::FAST; +using lsst::qserv::protojson::ScanInfo::Rating::FASTEST; +using lsst::qserv::protojson::ScanInfo::Rating::MEDIUM; +using lsst::qserv::protojson::ScanInfo::Rating::SLOW; using lsst::qserv::wbase::FileChannelShared; using lsst::qserv::wbase::SendChannel; using lsst::qserv::wbase::Task; +using lsst::qserv::wbase::UberJobData; using lsst::qserv::wconfig::WorkerConfig; using lsst::qserv::wcontrol::SqlConnMgr; using lsst::qserv::wdb::ChunkResourceMgr; @@ -81,97 +87,154 @@ auto workerCfg = lsst::qserv::wconfig::WorkerConfig::create(); std::vector locSendSharedPtrs; -/* &&& -Task::Ptr makeTask(std::shared_ptr tm, shared_ptr const& queries) { +lsst::qserv::protojson::ScanInfo::Ptr makeScanInfoFastest() { + auto info = lsst::qserv::protojson::ScanInfo::create(); + info->scanRating = FASTEST; + int const sRating = FAST; + string const db = "moose"; + string const table = "Object"; + bool const lockInMem = true; + info->infoTables.emplace_back(db, table, lockInMem, sRating); + return info; +} + +lsst::qserv::protojson::ScanInfo::Ptr makeScanInfoFast(string const& slowestTableName = string("")) { + auto info = lsst::qserv::protojson::ScanInfo::create(); + info->scanRating = FAST; + int const sRating = FAST; + string const db = "moose"; + string const table = "Object"; + bool const lockInMem = true; + info->infoTables.emplace_back(db, table, lockInMem, sRating); + info->sortTablesSlowestFirst(); + if (slowestTableName != string("")) { + info->infoTables[0].table = slowestTableName; + } + return info; +} + +lsst::qserv::protojson::ScanInfo::Ptr makeScanInfoMedium(string const& slowestTableName = string("")) { + auto info = lsst::qserv::protojson::ScanInfo::create(); + info->scanRating = MEDIUM; + string const db = "moose"; + bool const lockInMem = true; + info->infoTables.emplace_back(db, "Object", lockInMem, FAST); + info->infoTables.emplace_back(db, "Source", lockInMem, MEDIUM); + info->sortTablesSlowestFirst(); + if (slowestTableName != string("")) { + info->infoTables[0].table = slowestTableName; + } + return info; +} + +lsst::qserv::protojson::ScanInfo::Ptr makeScanInfoSlow(string const& slowestTableName = string("")) { + auto info = lsst::qserv::protojson::ScanInfo::create(); + info->scanRating = SLOW; + string const db = "moose"; + bool const lockInMem = true; + info->infoTables.emplace_back(db, "Object", lockInMem, FAST); + info->infoTables.emplace_back(db, "Source", lockInMem, MEDIUM); + info->infoTables.emplace_back(db, "ForcedSource", lockInMem, SLOW); + info->sortTablesSlowestFirst(); + if (slowestTableName != string("")) { + info->infoTables[0].table = slowestTableName; + } + return info; +} + +UberJobData::Ptr makeUberJobData(uint64_t queryId, + std::shared_ptr const& scanInfo, + bool scanInteractive) { + auto ujd = UberJobData::create(7, // UberJobId + "cz1", // czarName + 11, // czarId, + "aHost", // czarHost + 3333, // czarPort + queryId, + 0, // rowLimit + 5000000000, // maxTableSizeBytes + scanInfo, + scanInteractive, // scanInteractive + "worker_13", // workerId, + nullptr, // std::shared_ptr const& foreman + "whatever" // authKey + ); + return ujd; +} + +Task::Ptr makeTask(UberJobData::Ptr const& ujData, int jobId, int chunkId, int fragmentNumber, + size_t templateId, bool hasSubchunks, int subchunkId, + vector const& fragSubTables, + vector const& fragSubchunkIds, shared_ptr const& sc, + std::shared_ptr const& queryStats) { WorkerConfig::create(); - auto sendC = std::make_shared(); - auto sc = FileChannelShared::create(sendC, tm->czarid()); - locSendSharedPtrs.push_back(sc); - auto taskVect = Task::createTasks(tm, sc, crm, mySqlConfig, sqlConnMgr, queries); - Task::Ptr task = taskVect[0]; + string const db = ujData->getScanInfo()->infoTables[0].db; + int const attemptCount = 0; + Task::Ptr task = shared_ptr(new Task(ujData, jobId, attemptCount, chunkId, fragmentNumber, + templateId, hasSubchunks, subchunkId, db, fragSubTables, + fragSubchunkIds, sc, queryStats)); return task; } -*/ struct SchedulerFixture { SchedulerFixture(void) { counter = 20; } ~SchedulerFixture(void) {} - /* &&& Instead of using messages, make a Task::createTasksForUnitTest() function - void addSomeFragments(TaskMsgPtr const& t, int numberOfFragments) { - for (int i = 0; i < numberOfFragments; ++i) { - TaskMsg::Fragment* f = t->add_fragment(); - f->add_query("Hello, this is a query."); - f->mutable_subchunks()->add_id(100 + i); - f->set_resulttable("r_341"); - } - } - - - TaskMsgPtr newTaskMsg(int seq, lsst::qserv::QueryId qId, int jobId) { - TaskMsgPtr t = std::make_shared(); - t->set_queryid(qId); - t->set_jobid(jobId); - t->set_chunkid(seq); - t->set_czarid(1); - t->set_db("elephant"); - //&&&addSomeFragments(t, 3); - t->set_scaninteractive(false); - t->set_attemptcount(0); + Task::Ptr makeUTask(int seq, int jobId, UberJobData::Ptr const& ujData, + shared_ptr const& sc, + shared_ptr const& queries) { ++counter; + int const chunkId = seq; + int const fragmentNumber = 0; + size_t const templateId = 0; + bool const hasSubchunks = false; + int const subchunkId = 0; + vector fragSubTables; + vector fragSubchunkIds; + Task::Ptr t = makeTask(ujData, jobId, chunkId, fragmentNumber, templateId, hasSubchunks, subchunkId, + fragSubTables, fragSubchunkIds, sc, queries->getStats(ujData->getQueryId())); return t; } - TaskMsgPtr newTaskMsgSimple(int seq, lsst::qserv::QueryId qId, int jobId) { - TaskMsgPtr t = std::make_shared(); - t->set_queryid(qId); - t->set_jobid(jobId); - t->set_chunkid(seq); - t->set_czarid(1); - t->set_db("moose"); - t->set_scaninteractive(false); - t->set_attemptcount(0); - //&&&addSomeFragments(t, 1); - ++counter; - return t; - } - - TaskMsgPtr newTaskMsgScan(int seq, int priority, lsst::qserv::QueryId qId, int jobId, - std::string const& tableName = "whatever") { - auto taskMsg = newTaskMsg(seq, qId, jobId); - taskMsg->set_scanpriority(priority); - auto sTbl = taskMsg->add_scantable(); - sTbl->set_db("elephant"); - sTbl->set_table(tableName); - sTbl->set_scanrating(priority); - sTbl->set_lockinmemory(true); - return taskMsg; - } - - Task::Ptr queMsgWithChunkId(wsched::GroupScheduler& gs, int chunkId, lsst::qserv::QueryId qId, int jobId, + Task::Ptr queMsgWithChunkId(UberJobData::Ptr const& ujData, wsched::GroupScheduler& gs, int chunkId, + int jobId, shared_ptr const& sc, shared_ptr const& queries) { - Task::Ptr t = makeTask(newTaskMsg(chunkId, qId, jobId), queries); + int const fragmentNumber = 0; + size_t const templateId = 0; + bool const hasSubchunks = false; + int const subchunkId = 0; + vector fragSubTables; + vector fragSubchunkIds; + Task::Ptr t = makeTask(ujData, jobId, chunkId, fragmentNumber, templateId, hasSubchunks, subchunkId, + fragSubTables, fragSubchunkIds, sc, queries->getStats(ujData->getQueryId())); gs.queCmd(t); return t; } - */ int counter; }; BOOST_FIXTURE_TEST_SUITE(SchedulerSuite, SchedulerFixture) +/// examineAfter=0 Don't run _examineThread when 0 +/// deadAfter=1 Consider queries dead if they finished more than 1 second ago. +lsst::qserv::wpublish::QueriesAndChunks::Ptr setupQueries(int maxTasksBooted, int maxDarkTasks, + bool resetForTesting, int deadAfter = 1, + int examineAfter = 0) { + auto qac = lsst::qserv::wpublish::QueriesAndChunks::setupGlobal( + chrono::seconds(deadAfter), chrono::seconds(examineAfter), maxTasksBooted, maxDarkTasks, + resetForTesting); + return qac; +} + struct SchedFixture { - SchedFixture(double maxScanTimeFast, bool examinAllSleep) - : _maxScanTimeFast{maxScanTimeFast}, _examineAllSleep{examinAllSleep} { + SchedFixture(double maxScanTimeFast, lsst::qserv::wpublish::QueriesAndChunks::Ptr const& queries_) + : _maxScanTimeFast(maxScanTimeFast), queries(queries_) { setupQueriesBlend(); } ~SchedFixture() {} void setupQueriesBlend() { - queries = lsst::qserv::wpublish::QueriesAndChunks::setupGlobal( - std::chrono::seconds(1), std::chrono::seconds(_examineAllSleep), maxBootedC, maxDarkTasksC, - resetForTestingC); blend = std::make_shared("blendSched", queries, maxThreads, group, scanSlow, scanSchedulers); group->setDefaultPosition(0); @@ -182,42 +245,45 @@ struct SchedFixture { queries->setRequiredTasksCompleted(1); // Make it easy to set a baseline. } - int const fastest = lsst::qserv::protojson::ScanInfo::Rating::FASTEST; - int const fast = lsst::qserv::protojson::ScanInfo::Rating::FAST; - int const medium = lsst::qserv::protojson::ScanInfo::Rating::MEDIUM; - int const slow = lsst::qserv::protojson::ScanInfo::Rating::SLOW; - lsst::qserv::QueryId qIdInc{1}; - int maxThreads{9}; int maxActiveChunks{20}; int priority{2}; private: double _maxScanTimeFast{oneHr}; ///< Don't hit time limit in tests. - int _examineAllSleep{0}; ///< Don't run _examineThread when 0 public: wsched::GroupScheduler::Ptr group{ std::make_shared("GroupSched", maxThreads, 2, 3, priority++)}; wsched::ScanScheduler::Ptr scanSlow{std::make_shared( - "ScanSlow", maxThreads, 2, priority++, maxActiveChunks, medium + 1, slow, oneHr)}; + "ScanSlow", maxThreads, 2, priority++, maxActiveChunks, MEDIUM + 1, SLOW, oneHr)}; wsched::ScanScheduler::Ptr scanMed{std::make_shared( - "ScanMed", maxThreads, 2, priority++, maxActiveChunks, fast + 1, medium, oneHr)}; + "ScanMed", maxThreads, 2, priority++, maxActiveChunks, FAST + 1, MEDIUM, oneHr)}; wsched::ScanScheduler::Ptr scanFast{std::make_shared( - "ScanFast", maxThreads, 3, priority++, maxActiveChunks, fastest, fast, _maxScanTimeFast)}; + "ScanFast", maxThreads, 3, priority++, maxActiveChunks, FASTEST, FAST, _maxScanTimeFast)}; std::vector scanSchedulers{scanFast, scanMed}; lsst::qserv::wpublish::QueriesAndChunks::Ptr queries; wsched::BlendScheduler::Ptr blend; }; +void logCmd(lsst::qserv::util::Command::Ptr const& cmd, std::string const& note) { + if (cmd == nullptr) + LOGS(_log, LOG_LVL_WARN, note << " null"); + else + LOGS(_log, LOG_LVL_WARN, note << ":" << cmd->dump()); +} + // TODO: DM-33302 replace this test case BOOST_AUTO_TEST_CASE(Grouping) { -#if 0 // &&& fix and re-enable - SchedFixture f(60.0, 1); // Values to keep QueriesAndChunk from triggering. - LOGS(_log, LOG_LVL_DEBUG, "Test_case grouping"); + + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); // Values to keep QueriesAndChunk from triggering. + // Test grouping by chunkId. Max entries added to a single group set to 3. wsched::GroupScheduler gs{"GroupSchedA", 100, 0, 3, 0}; // chunk Ids @@ -230,25 +296,52 @@ BOOST_AUTO_TEST_CASE(Grouping) { BOOST_CHECK(gs.ready() == false); lsst::qserv::QueryId qIdInc = 1; - Task::Ptr a1 = queMsgWithChunkId(gs, a, qIdInc++, 0, f.queries); + // Either FASTEST scan rating or scanInteractive = true should make the scan interactive. + auto scanInfoFastest = makeScanInfoFastest(); + auto scanInfoFast = makeScanInfoFast(); + bool const scanInteractive = true; + auto ujData_a1 = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + shared_ptr sc = nullptr; + Task::Ptr a1 = queMsgWithChunkId(ujData_a1, gs, a, 0, sc, fixt.queries); BOOST_CHECK(gs.empty() == false); BOOST_CHECK(gs.ready() == true); - Task::Ptr b1 = queMsgWithChunkId(gs, b, qIdInc++, 0, f.queries); - Task::Ptr c1 = queMsgWithChunkId(gs, c, qIdInc++, 0, f.queries); - Task::Ptr b2 = queMsgWithChunkId(gs, b, qIdInc++, 0, f.queries); - Task::Ptr b3 = queMsgWithChunkId(gs, b, qIdInc++, 0, f.queries); - Task::Ptr b4 = queMsgWithChunkId(gs, b, qIdInc++, 0, f.queries); - Task::Ptr a2 = queMsgWithChunkId(gs, a, qIdInc++, 0, f.queries); - Task::Ptr a3 = queMsgWithChunkId(gs, a, qIdInc++, 0, f.queries); - Task::Ptr b5 = queMsgWithChunkId(gs, b, qIdInc++, 0, f.queries); - Task::Ptr d1 = queMsgWithChunkId(gs, d, qIdInc++, 0, f.queries); + auto b1Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr b1 = queMsgWithChunkId(b1Ujd, gs, b, 0, sc, fixt.queries); + + auto c1Ujd = makeUberJobData(qIdInc++, scanInfoFast, scanInteractive); + Task::Ptr c1 = queMsgWithChunkId(c1Ujd, gs, c, 0, sc, fixt.queries); + + auto b2Ujd = makeUberJobData(qIdInc++, scanInfoFastest, false); + Task::Ptr b2 = queMsgWithChunkId(b2Ujd, gs, b, 0, sc, fixt.queries); + + auto b3Ujd = makeUberJobData(qIdInc++, scanInfoFast, scanInteractive); + Task::Ptr b3 = queMsgWithChunkId(b3Ujd, gs, b, 0, sc, fixt.queries); + + auto b4Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr b4 = queMsgWithChunkId(b4Ujd, gs, b, 0, sc, fixt.queries); + + auto a2Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr a2 = queMsgWithChunkId(a2Ujd, gs, a, 0, sc, fixt.queries); + + auto a3Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr a3 = queMsgWithChunkId(a3Ujd, gs, a, 0, sc, fixt.queries); + + auto b5Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr b5 = queMsgWithChunkId(b5Ujd, gs, b, 0, sc, fixt.queries); + + auto d1Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr d1 = queMsgWithChunkId(d1Ujd, gs, d, 0, sc, fixt.queries); + BOOST_CHECK(gs.getSize() == 5); BOOST_CHECK(gs.ready() == true); + // Should get all the first 3 'a' commands in order auto aa1 = gs.getCmd(false); auto aa2 = gs.getCmd(false); - Task::Ptr a4 = queMsgWithChunkId(gs, a, qIdInc++, 0, f.queries); // this should get its own group + auto a4Ujd = makeUberJobData(qIdInc++, scanInfoFastest, scanInteractive); + Task::Ptr a4 = queMsgWithChunkId(a4Ujd, gs, a, 0, sc, fixt.queries); // this should get its own group + auto aa3 = gs.getCmd(false); BOOST_CHECK(a1.get() == aa1.get()); BOOST_CHECK(a2.get() == aa2.get()); @@ -295,22 +388,32 @@ BOOST_AUTO_TEST_CASE(Grouping) { BOOST_CHECK(gs.getInFlight() == 10); BOOST_CHECK(gs.ready() == false); BOOST_CHECK(gs.empty() == true); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(GroupMaxThread) { -#if 0 // &&& fix and re-enable // Test that maxThreads is meaningful. LOGS(_log, LOG_LVL_WARN, "Test_case GroupMaxThread"); - auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, - maxDarkTasksC, resetForTestingC); + auto scanInfo = makeScanInfoFastest(); + bool const scanInteractive = true; + shared_ptr sc = nullptr; + + auto queries = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, 1, 300); wsched::GroupScheduler gs{"GroupSchedB", 3, 0, 100, 0}; lsst::qserv::QueryId qIdInc = 1; + int a = 42; - Task::Ptr a1 = queMsgWithChunkId(gs, a, qIdInc++, 0, queries); - Task::Ptr a2 = queMsgWithChunkId(gs, a, qIdInc++, 0, queries); - Task::Ptr a3 = queMsgWithChunkId(gs, a, qIdInc++, 0, queries); - Task::Ptr a4 = queMsgWithChunkId(gs, a, qIdInc++, 0, queries); + auto a1Ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a1 = queMsgWithChunkId(a1Ujd, gs, a, 0, sc, queries); + + auto a2Ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a2 = queMsgWithChunkId(a2Ujd, gs, a, 0, sc, queries); + + auto a3Ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a3 = queMsgWithChunkId(a3Ujd, gs, a, 0, sc, queries); + + auto a4Ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a4 = queMsgWithChunkId(a4Ujd, gs, a, 0, sc, queries); + BOOST_CHECK(gs.ready() == true); auto aa1 = gs.getCmd(false); BOOST_CHECK(a1.get() == aa1.get()); @@ -330,37 +433,37 @@ BOOST_AUTO_TEST_CASE(GroupMaxThread) { auto aa4 = gs.getCmd(false); BOOST_CHECK(a4.get() == aa4.get()); BOOST_CHECK(gs.ready() == false); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(ScanScheduleTest) { -#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case ScanScheduleTest"); - auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, - maxDarkTasksC, resetForTestingC); + + auto queries = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, 1, 300); wsched::ScanScheduler sched{"ScanSchedA", 2, 1, 0, 20, 0, 100, oneHr}; + auto scanInfo = makeScanInfoFast(); + bool const scanInteractive = true; + shared_ptr sc = nullptr; lsst::qserv::QueryId qIdInc = 1; - // Test ready state as Tasks added and removed. BOOST_CHECK(sched.ready() == false); - Task::Ptr a38 = makeTask(newTaskMsgScan(38, 0, qIdInc++, 0), queries); + auto ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a38 = makeUTask(38, 0, ujd, sc, queries); sched.queCmd(a38); // Calling read swaps active and pending heaps, putting a38 at the top of the active. BOOST_CHECK(sched.ready() == true); - Task::Ptr a40 = makeTask(newTaskMsgScan(40, 0, qIdInc++, 0), queries); // goes on active + ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a40 = makeUTask(40, 0, ujd, sc, queries); // goes on active sched.queCmd(a40); - // TODO: This needs to be evaluated and removed. - // Making a non-scan message so MemManNone will grant it an empty Handle - Task::Ptr b41 = makeTask(newTaskMsg(41, qIdInc++, 0), queries); // goes on active + ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr b41 = makeUTask(41, 0, ujd, sc, queries); // goes on active sched.queCmd(b41); - // TODO: This needs to be evaluated and removed. - // Making a non-scan message so MemManNone will grant it an empty Handle - Task::Ptr a33 = makeTask(newTaskMsg(33, qIdInc++, 0), queries); // goes on pending. + ujd = makeUberJobData(qIdInc++, scanInfo, scanInteractive); + Task::Ptr a33 = makeUTask(33, 0, ujd, sc, queries); // goes on pending. sched.queCmd(a33); BOOST_CHECK(sched.ready() == true); @@ -396,292 +499,325 @@ BOOST_AUTO_TEST_CASE(ScanScheduleTest) { sched.commandFinish(tsk1); BOOST_CHECK(sched.getInFlight() == 0); BOOST_CHECK(sched.ready() == false); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleTest) { -#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case BlendScheduleTest"); // Test that space is appropriately reserved for each scheduler as Tasks are started and finished. // TODO: This needs to be evaluated and removed. // In this case, memMan->lock(..) always returns true (really HandleType::ISEMPTY). // ChunkIds matter as they control the order Tasks come off individual schedulers. - SchedFixture f(60.0, 1); // Values to keep QueriesAndChunk from triggering. - - BOOST_CHECK(f.blend->ready() == false); - BOOST_CHECK(f.blend->calcAvailableTheads() == 5); + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); // Values to keep QueriesAndChunk from triggering. + + auto scanInfoFastest = makeScanInfoFastest(); + auto scanInfoFast = makeScanInfoFast(); + auto scanInfoMedium = makeScanInfoMedium(); + auto scanInfoSlow = makeScanInfoSlow(); + bool const scanInteractiveT = true; + bool const scanInteractiveF = false; + shared_ptr sc = nullptr; + + BOOST_CHECK(fixt.blend->ready() == false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 5); // Put one message on each scheduler except ScanFast, which gets 2. LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-1 add Tasks"); - Task::Ptr g1 = makeTask(newTaskMsgSimple(40, f.qIdInc++, 0), f.queries); - f.blend->queCmd(g1); - BOOST_CHECK(f.group->getSize() == 1); - BOOST_CHECK(f.blend->ready() == true); - - auto taskMsg = newTaskMsgScan(27, lsst::qserv::proto::ScanInfo::Rating::FAST, f.qIdInc++, 0); - Task::Ptr sF1 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sF1); - BOOST_CHECK(f.scanFast->getSize() == 1); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(40, lsst::qserv::proto::ScanInfo::Rating::FAST, f.qIdInc++, 0); - Task::Ptr sF2 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sF2); - BOOST_CHECK(f.scanFast->getSize() == 2); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(34, lsst::qserv::proto::ScanInfo::Rating::SLOW, f.qIdInc++, 0); - Task::Ptr sS1 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sS1); - BOOST_CHECK(f.scanSlow->getSize() == 1); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(31, lsst::qserv::proto::ScanInfo::Rating::MEDIUM, f.qIdInc++, 0); - Task::Ptr sM1 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sM1); - BOOST_CHECK(f.scanMed->getSize() == 1); - BOOST_CHECK(f.blend->ready() == true); - - BOOST_CHECK(f.blend->getSize() == 5); - BOOST_CHECK(f.blend->calcAvailableTheads() == 5); + auto ujd = makeUberJobData(fixt.qIdInc++, scanInfoFastest, scanInteractiveT); + Task::Ptr g1 = makeUTask(40, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(g1); + BOOST_CHECK(fixt.group->getSize() == 1); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoFast, scanInteractiveF); + Task::Ptr sF1 = makeUTask(27, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sF1); + BOOST_CHECK(fixt.scanFast->getSize() == 1); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoFast, scanInteractiveF); + Task::Ptr sF2 = makeUTask(40, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sF2); + BOOST_CHECK(fixt.scanFast->getSize() == 2); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoSlow, scanInteractiveF); + Task::Ptr sS1 = makeUTask(34, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sS1); + BOOST_CHECK(fixt.scanSlow->getSize() == 1); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoMedium, scanInteractiveF); + Task::Ptr sM1 = makeUTask(31, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sM1); + BOOST_CHECK(fixt.scanMed->getSize() == 1); + BOOST_CHECK(fixt.blend->ready() == true); + + BOOST_CHECK(fixt.blend->getSize() == 5); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 5); // Start all the Tasks. LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-1 start all tasks"); // Tasks should come out in order of scheduler priority. - auto og1 = f.blend->getCmd(false); + auto og1 = fixt.blend->getCmd(false); BOOST_CHECK(og1.get() == g1.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 4); - auto osF1 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 4); + auto osF1 = fixt.blend->getCmd(false); BOOST_CHECK(osF1.get() == sF1.get()); // sF1 has lower chunkId than sF2 - BOOST_CHECK(f.blend->calcAvailableTheads() == 3); - auto osF2 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 3); + auto osF2 = fixt.blend->getCmd(false); BOOST_CHECK(osF2.get() == sF2.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 2); - auto osM1 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 2); + auto osM1 = fixt.blend->getCmd(false); BOOST_CHECK(osM1.get() == sM1.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 1); - auto osS1 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 1); + auto osS1 = fixt.blend->getCmd(false); BOOST_CHECK(osS1.get() == sS1.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - BOOST_CHECK(f.blend->getSize() == 0); - BOOST_CHECK(f.blend->ready() == false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + BOOST_CHECK(fixt.blend->getSize() == 0); + BOOST_CHECK(fixt.blend->ready() == false); // All threads should now be in use or reserved, should be able to start one // Task for each scheduler but second Task should remain on queue. - Task::Ptr g2 = makeTask(newTaskMsgSimple(41, f.qIdInc++, 0), f.queries); - f.blend->queCmd(g2); - BOOST_CHECK(f.group->getSize() == 1); - BOOST_CHECK(f.blend->getSize() == 1); - BOOST_CHECK(f.blend->ready() == true); - - Task::Ptr g3 = makeTask(newTaskMsgSimple(12, f.qIdInc++, 0), f.queries); - f.blend->queCmd(g3); - BOOST_CHECK(f.group->getSize() == 2); - BOOST_CHECK(f.blend->getSize() == 2); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(70, lsst::qserv::proto::ScanInfo::Rating::FAST, f.qIdInc++, 0); - Task::Ptr sF3 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sF3); - BOOST_CHECK(f.scanFast->getSize() == 1); - BOOST_CHECK(f.blend->getSize() == 3); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(72, lsst::qserv::proto::ScanInfo::Rating::FAST, f.qIdInc++, 0); - Task::Ptr sF4 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sF4); - BOOST_CHECK(f.scanFast->getSize() == 2); - BOOST_CHECK(f.blend->getSize() == 4); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(13, lsst::qserv::proto::ScanInfo::Rating::MEDIUM, f.qIdInc++, 0); - Task::Ptr sM2 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sM2); - BOOST_CHECK(f.scanMed->getSize() == 1); - BOOST_CHECK(f.blend->getSize() == 5); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(15, lsst::qserv::proto::ScanInfo::Rating::MEDIUM, f.qIdInc++, 0); - Task::Ptr sM3 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sM3); - BOOST_CHECK(f.scanMed->getSize() == 2); - BOOST_CHECK(f.blend->getSize() == 6); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(5, lsst::qserv::proto::ScanInfo::Rating::SLOW, f.qIdInc++, 0); - Task::Ptr sS2 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sS2); - BOOST_CHECK(f.scanSlow->getSize() == 1); - BOOST_CHECK(f.blend->getSize() == 7); - BOOST_CHECK(f.blend->ready() == true); - - taskMsg = newTaskMsgScan(6, lsst::qserv::proto::ScanInfo::Rating::SLOW, f.qIdInc++, 0); - Task::Ptr sS3 = makeTask(taskMsg, f.queries); - f.blend->queCmd(sS3); - BOOST_CHECK(f.scanSlow->getSize() == 2); - BOOST_CHECK(f.blend->getSize() == 8); - BOOST_CHECK(f.blend->ready() == true); + ujd = makeUberJobData(fixt.qIdInc++, scanInfoFastest, scanInteractiveT); + Task::Ptr g2 = makeUTask(41, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(g2); + BOOST_CHECK(fixt.group->getSize() == 1); + BOOST_CHECK(fixt.blend->getSize() == 1); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoFastest, scanInteractiveT); + Task::Ptr g3 = makeUTask(12, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(g3); + BOOST_CHECK(fixt.group->getSize() == 2); + BOOST_CHECK(fixt.blend->getSize() == 2); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoFast, scanInteractiveF); + Task::Ptr sF3 = makeUTask(70, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sF3); + BOOST_CHECK(fixt.scanFast->getSize() == 1); + BOOST_CHECK(fixt.blend->getSize() == 3); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoFast, scanInteractiveF); + Task::Ptr sF4 = makeUTask(72, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sF4); + BOOST_CHECK(fixt.scanFast->getSize() == 2); + BOOST_CHECK(fixt.blend->getSize() == 4); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoMedium, scanInteractiveF); + Task::Ptr sM2 = makeUTask(13, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sM2); + BOOST_CHECK(fixt.scanMed->getSize() == 1); + BOOST_CHECK(fixt.blend->getSize() == 5); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoMedium, scanInteractiveF); + Task::Ptr sM3 = makeUTask(15, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sM3); + BOOST_CHECK(fixt.scanMed->getSize() == 2); + BOOST_CHECK(fixt.blend->getSize() == 6); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoSlow, scanInteractiveF); + Task::Ptr sS2 = makeUTask(5, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sS2); + BOOST_CHECK(fixt.scanSlow->getSize() == 1); + BOOST_CHECK(fixt.blend->getSize() == 7); + BOOST_CHECK(fixt.blend->ready() == true); + + ujd = makeUberJobData(fixt.qIdInc++, scanInfoSlow, scanInteractiveF); + Task::Ptr sS3 = makeUTask(6, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(sS3); + BOOST_CHECK(fixt.scanSlow->getSize() == 2); + BOOST_CHECK(fixt.blend->getSize() == 8); + BOOST_CHECK(fixt.blend->ready() == true); // Expect 1 group, 1 fast, 1 medium, and 1 slow in that order - auto og2 = f.blend->getCmd(false); + auto og2 = fixt.blend->getCmd(false); BOOST_CHECK(og2.get() == g2.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - auto osF3 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + BOOST_CHECK(fixt.blend->ready() == true); + auto osF3 = fixt.blend->getCmd(false); BOOST_CHECK(osF3.get() == sF3.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - BOOST_CHECK(f.blend->ready() == true); - auto osM2 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + BOOST_CHECK(fixt.blend->ready() == true); + auto osM2 = fixt.blend->getCmd(false); BOOST_CHECK(osM2.get() == sM2.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - BOOST_CHECK(f.blend->ready() == true); - auto osS2 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + BOOST_CHECK(fixt.blend->ready() == true); + auto osS2 = fixt.blend->getCmd(false); BOOST_CHECK(osS2.get() == sS2.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - BOOST_CHECK(f.blend->getSize() == 4); - BOOST_CHECK(f.blend->ready() == false); // all threads in use + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + BOOST_CHECK(fixt.blend->getSize() == 4); + BOOST_CHECK(fixt.blend->ready() == false); // all threads in use // Finishing a fast Task should allow the last fast Task to go. LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-1 call commandFinish"); - f.blend->commandFinish(osF3); - auto osF4 = f.blend->getCmd(false); + fixt.blend->commandFinish(osF3); + auto osF4 = fixt.blend->getCmd(false); BOOST_CHECK(osF4.get() == sF4.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - BOOST_CHECK(f.blend->ready() == false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + BOOST_CHECK(fixt.blend->ready() == false); // Finishing 2 fast Tasks should allow a group Task to go. - f.blend->commandFinish(osF1); - BOOST_CHECK(f.blend->calcAvailableTheads() == 0); - f.blend->commandFinish(osF2); - BOOST_CHECK(f.blend->calcAvailableTheads() == 1); - auto og3 = f.blend->getCmd(false); + fixt.blend->commandFinish(osF1); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 0); + fixt.blend->commandFinish(osF2); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 1); + auto og3 = fixt.blend->getCmd(false); BOOST_CHECK(og3.get() == g3.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 1); - BOOST_CHECK(f.blend->ready() == false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 1); + BOOST_CHECK(fixt.blend->ready() == false); // Finishing the last fast Task should let a medium Task go. - f.blend->commandFinish(osF4); - BOOST_CHECK(f.blend->calcAvailableTheads() == 2); - auto osM3 = f.blend->getCmd(false); + fixt.blend->commandFinish(osF4); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 2); + auto osM3 = fixt.blend->getCmd(false); BOOST_CHECK(osM3.get() == sM3.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 2); - BOOST_CHECK(f.blend->ready() == false); - BOOST_CHECK(f.blend->getCmd(false) == nullptr); - - // Finishing a group Task should allow a slow Task to got (only remaining Task) - BOOST_CHECK(f.blend->getSize() == 1); - f.blend->commandFinish(og1); - BOOST_CHECK(f.blend->calcAvailableTheads() == 2); - auto osS3 = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 2); + BOOST_CHECK(fixt.blend->ready() == false); + BOOST_CHECK(fixt.blend->getCmd(false) == nullptr); + + // Finishing a group Task should allow a slow Task to go (only remaining Task) + BOOST_CHECK(fixt.blend->getSize() == 1); + fixt.blend->commandFinish(og1); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 2); + auto osS3 = fixt.blend->getCmd(false); BOOST_CHECK(osS3.get() == sS3.get()); - BOOST_CHECK(f.blend->calcAvailableTheads() == 2); - BOOST_CHECK(f.blend->getSize() == 0); - BOOST_CHECK(f.blend->ready() == false); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 2); + BOOST_CHECK(fixt.blend->getSize() == 0); + BOOST_CHECK(fixt.blend->ready() == false); // Close out all tasks and check counts. LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-1 close out all Tasks"); - f.blend->commandFinish(og2); - BOOST_CHECK(f.blend->calcAvailableTheads() == 2); - BOOST_CHECK(f.blend->getInFlight() == 7); - f.blend->commandFinish(og3); - BOOST_CHECK(f.blend->calcAvailableTheads() == 3); - BOOST_CHECK(f.blend->getInFlight() == 6); - f.blend->commandFinish(osM1); - BOOST_CHECK(f.blend->calcAvailableTheads() == 3); - BOOST_CHECK(f.blend->getInFlight() == 5); - f.blend->commandFinish(osM2); - BOOST_CHECK(f.blend->calcAvailableTheads() == 3); - f.blend->commandFinish(osM3); - BOOST_CHECK(f.blend->calcAvailableTheads() == 4); - f.blend->commandFinish(osS1); - BOOST_CHECK(f.blend->calcAvailableTheads() == 4); - f.blend->commandFinish(osS2); - BOOST_CHECK(f.blend->calcAvailableTheads() == 4); - f.blend->commandFinish(osS3); - BOOST_CHECK(f.blend->calcAvailableTheads() == 5); - BOOST_CHECK(f.blend->getInFlight() == 0); + fixt.blend->commandFinish(og2); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 2); + BOOST_CHECK(fixt.blend->getInFlight() == 7); + fixt.blend->commandFinish(og3); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 3); + BOOST_CHECK(fixt.blend->getInFlight() == 6); + fixt.blend->commandFinish(osM1); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 3); + BOOST_CHECK(fixt.blend->getInFlight() == 5); + fixt.blend->commandFinish(osM2); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 3); + fixt.blend->commandFinish(osM3); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 4); + fixt.blend->commandFinish(osS1); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 4); + fixt.blend->commandFinish(osS2); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 4); + fixt.blend->commandFinish(osS3); + BOOST_CHECK(fixt.blend->calcAvailableTheads() == 5); + BOOST_CHECK(fixt.blend->getInFlight() == 0); LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-1 done"); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleThreadLimitingTest) { -#if 0 // &&& fix and re-enable LOGS(_log, LOG_LVL_DEBUG, "Test_case BlendScheduleThreadLimitingTest"); - SchedFixture f(60.0, 1); // Values to keep QueriesAndChunk from triggering. + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); // Values to keep QueriesAndChunk from triggering. + + auto scanInfoFastest = makeScanInfoFastest(); + auto scanInfoFast = makeScanInfoFast(); + auto scanInfoMedium = makeScanInfoMedium(); + auto scanInfoSlow = makeScanInfoSlow(); + bool const scanInteractiveT = true; + bool const scanInteractiveF = false; + shared_ptr sc = nullptr; + // Test that only 6 threads can be started on a single ScanScheduler // This leaves 3 threads available, 1 for each other scheduler. - BOOST_CHECK(f.blend->ready() == false); + BOOST_CHECK(fixt.blend->ready() == false); std::vector scanTasks; for (int j = 0; j < 7; ++j) { - auto tsk = makeTask(newTaskMsgScan(j, lsst::qserv::proto::ScanInfo::Rating::MEDIUM, f.qIdInc++, 0), - f.queries); - f.blend->queCmd(tsk); + auto ujd = makeUberJobData(fixt.qIdInc++, scanInfoMedium, scanInteractiveF); + auto tsk = makeUTask(j, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(tsk); if (j < 6) { - BOOST_CHECK(f.blend->ready() == true); - auto cmd = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->ready() == true); + auto cmd = fixt.blend->getCmd(false); BOOST_CHECK(cmd != nullptr); auto task = std::dynamic_pointer_cast(cmd); scanTasks.push_back(task); } if (j == 6) { - BOOST_CHECK(f.blend->ready() == false); - BOOST_CHECK(f.blend->getCmd(false) == nullptr); + BOOST_CHECK(fixt.blend->ready() == false); + BOOST_CHECK(fixt.blend->getCmd(false) == nullptr); } } { // Finishing one task should allow the 7th one to run. - f.blend->commandFinish(scanTasks[0]); - BOOST_CHECK(f.blend->ready() == true); - auto cmd = f.blend->getCmd(false); + fixt.blend->commandFinish(scanTasks[0]); + BOOST_CHECK(fixt.blend->ready() == true); + auto cmd = fixt.blend->getCmd(false); BOOST_CHECK(cmd != nullptr); auto task = std::dynamic_pointer_cast(cmd); scanTasks.push_back(task); } // Finish all the scanTasks, scanTasks[0] is already finished. - for (int j = 1; j < 7; ++j) f.blend->commandFinish(scanTasks[j]); - BOOST_CHECK(f.blend->getInFlight() == 0); - BOOST_CHECK(f.blend->ready() == false); + for (int j = 1; j < 7; ++j) fixt.blend->commandFinish(scanTasks[j]); + BOOST_CHECK(fixt.blend->getInFlight() == 0); + BOOST_CHECK(fixt.blend->ready() == false); // Test that only 6 threads can be started on a single GroupScheduler // This leaves 3 threads available, 1 for each other scheduler. std::vector groupTasks; for (int j = 0; j < 7; ++j) { - f.blend->queCmd(makeTask(newTaskMsg(j, f.qIdInc++, 0), f.queries)); + auto ujd = makeUberJobData(fixt.qIdInc++, scanInfoFastest, scanInteractiveT); + auto tsk = makeUTask(j, 0, ujd, sc, fixt.queries); + fixt.blend->queCmd(tsk); if (j < 6) { - BOOST_CHECK(f.blend->ready() == true); - auto cmd = f.blend->getCmd(false); + BOOST_CHECK(fixt.blend->ready() == true); + auto cmd = fixt.blend->getCmd(false); BOOST_CHECK(cmd != nullptr); auto task = std::dynamic_pointer_cast(cmd); groupTasks.push_back(task); } if (j == 6) { - BOOST_CHECK(f.blend->ready() == false); - BOOST_CHECK(f.blend->getCmd(false) == nullptr); + BOOST_CHECK(fixt.blend->ready() == false); + BOOST_CHECK(fixt.blend->getCmd(false) == nullptr); } } { // Finishing one task should allow the 7th one to run. - f.blend->commandFinish(groupTasks[0]); - BOOST_CHECK(f.blend->ready() == true); - auto cmd = f.blend->getCmd(false); + fixt.blend->commandFinish(groupTasks[0]); + BOOST_CHECK(fixt.blend->ready() == true); + auto cmd = fixt.blend->getCmd(false); BOOST_CHECK(cmd != nullptr); auto task = std::dynamic_pointer_cast(cmd); groupTasks.push_back(task); } // Finish all the groupTasks, groupTasks[0] is already finished. - for (int j = 1; j < 7; ++j) f.blend->commandFinish(groupTasks[j]); - BOOST_CHECK(f.blend->getInFlight() == 0); - BOOST_CHECK(f.blend->ready() == false); + for (int j = 1; j < 7; ++j) fixt.blend->commandFinish(groupTasks[j]); + BOOST_CHECK(fixt.blend->getInFlight() == 0); + BOOST_CHECK(fixt.blend->ready() == false); LOGS(_log, LOG_LVL_DEBUG, "BlendScheduleTest-2 done"); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(BlendScheduleQueryRemovalTest) { -#if 0 // &&& fix and re-enable // Test that space is appropriately reserved for each scheduler as Tasks are started and finished. - // TODO: This needs to be evaluated and removed. // In this case, memMan->lock(..) always returns true (really HandleType::ISEMPTY). // ChunkIds matter as they control the order Tasks come off individual schedulers. - SchedFixture f(60.0, 1); // Values to keep QueriesAndChunk from triggering. + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); // Values to keep QueriesAndChunk from triggering. + + auto scanInfoFastest = makeScanInfoFastest(); + auto scanInfoFast = makeScanInfoFast(); + auto scanInfoMedium = makeScanInfoMedium(); + auto scanInfoSlow = makeScanInfoSlow(); + bool const scanInteractiveF = false; + shared_ptr sc = nullptr; + LOGS(_log, LOG_LVL_DEBUG, "Test_case BlendScheduleQueryRemovalTest"); // Add two queries to scanFast scheduler and then move one query to scanSlow. int startChunk = 70; @@ -690,30 +826,32 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryRemovalTest) { unsigned int jobsB = jobs; std::vector queryATasks; std::vector queryBTasks; - lsst::qserv::QueryId qIdA = f.qIdInc++; - lsst::qserv::QueryId qIdB = f.qIdInc++; + lsst::qserv::QueryId qIdA = fixt.qIdInc++; + lsst::qserv::QueryId qIdB = fixt.qIdInc++; { int jobId = 0; int chunkId = startChunk; + auto ujdA = makeUberJobData(qIdA, scanInfoFast, scanInteractiveF); + auto ujdB = makeUberJobData(qIdB, scanInfoFast, scanInteractiveF); for (unsigned int j = 0; j < jobs; ++j) { - auto taskMsg = newTaskMsgScan(chunkId, lsst::qserv::proto::ScanInfo::Rating::FAST, qIdA, jobId); - Task::Ptr mv = makeTask(taskMsg, f.queries); + Task::Ptr mv = makeUTask(chunkId, jobId, ujdA, sc, fixt.queries); queryATasks.push_back(mv); - f.queries->addTask(mv); - f.blend->queCmd(mv); - taskMsg = newTaskMsgScan(chunkId++, lsst::qserv::proto::ScanInfo::Rating::FAST, qIdB, jobId++); - mv = makeTask(taskMsg, f.queries); + fixt.queries->addTask(mv); + fixt.blend->queCmd(mv); + mv = makeUTask(chunkId, jobId, ujdB, sc, fixt.queries); queryBTasks.push_back(mv); - f.queries->addTask(mv); - f.blend->queCmd(mv); + fixt.queries->addTask(mv); + fixt.blend->queCmd(mv); + chunkId++; + jobId++; } } - BOOST_CHECK(f.scanFast->getSize() == jobs * 2); - BOOST_CHECK(f.scanSlow->getSize() == 0); + BOOST_CHECK(fixt.scanFast->getSize() == jobs * 2); + BOOST_CHECK(fixt.scanSlow->getSize() == 0); // This should run 1 job from one of the queries, but there are no guarantees about which one. // This is to test that moveUserQuery behaves appropriately for running Tasks. - auto poppedTask = f.blend->getCmd(false); + auto poppedTask = fixt.blend->getCmd(false); bool poppedFromA = false; for (auto const& tk : queryATasks) { if (tk == poppedTask) { @@ -726,58 +864,68 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryRemovalTest) { else --jobsB; - f.blend->moveUserQuery(qIdA, f.scanFast, f.scanSlow); // move query qIdA to scanSlow. - LOGS(_log, LOG_LVL_DEBUG, "fastSize=" << f.scanFast->getSize() << " slowSize=" << f.scanSlow->getSize()); - BOOST_CHECK(f.scanFast->getSize() == jobsB); - BOOST_CHECK(f.scanSlow->getSize() == jobsA); + fixt.blend->moveUserQuery(qIdA, fixt.scanFast, fixt.scanSlow); // move query qIdA to scanSlow. + LOGS(_log, LOG_LVL_DEBUG, + "fastSize=" << fixt.scanFast->getSize() << " slowSize=" << fixt.scanSlow->getSize()); + BOOST_CHECK(fixt.scanFast->getSize() == jobsB); + BOOST_CHECK(fixt.scanSlow->getSize() == jobsA); // Can't use queryATasks[0] for this as it was popped from the queue before the move. auto taskFromA = queryATasks[1]; auto schedForA = std::dynamic_pointer_cast(taskFromA->getTaskScheduler()); LOGS(_log, LOG_LVL_DEBUG, "taskFromA=" << taskFromA->getIdStr() << " sched=" << schedForA->getName()); - BOOST_CHECK(schedForA == f.scanSlow); -#endif // &&& fix and re-enable + BOOST_CHECK(schedForA == fixt.scanSlow); } BOOST_AUTO_TEST_CASE(BlendScheduleQueryBootTaskTest) { -#if 0 // &&& fix and re-enable // Test if a task is removed if it takes takes too long. // Give the user query 0.1 seconds to run and run it for a second, it should get removed. double tenthOfSecInMinutes = 1.0 / 600.0; // task - SchedFixture f(tenthOfSecInMinutes, 1); // sleep 1 second then check if tasks took too long + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(tenthOfSecInMinutes, qac); // Values to keep QueriesAndChunk from triggering. + + auto scanInfoFastest = makeScanInfoFastest(); + auto scanInfoFast = makeScanInfoFast(); + auto scanInfoMedium = makeScanInfoMedium(); + auto scanInfoSlow = makeScanInfoSlow(); + bool const scanInteractiveF = false; + shared_ptr sc = nullptr; LOGS(_log, LOG_LVL_DEBUG, "Test_case BlendScheduleQueryBootTaskTest"); // Create a thread pool to run task - auto pool = lsst::qserv::util::ThreadPool::newThreadPool(20, 1000, f.blend); + auto pool = lsst::qserv::util::ThreadPool::newThreadPool(20, 1000, fixt.blend); // Create fake data - one query to get a baseline time, another to take too long. // IMPORTANT: the "fast" taskl is needed to establish the baseline in QueriesAndChunks. // Otherwise the next task (the one which is going to be booted from its scheduler) // won't be booted. - int qid = 5; - auto taskMsg = newTaskMsgScan(27, lsst::qserv::proto::ScanInfo::Rating::FAST, qid++, 0); - Task::Ptr task = makeTask(taskMsg, f.queries); + int const qidA = 5; + int const qidB = 6; + auto ujd = makeUberJobData(qidA, scanInfoFast, scanInteractiveF); + Task::Ptr task = makeUTask(27, 0, ujd, sc, fixt.queries); std::atomic running{false}; - auto fastFunc = [&running, &task, queriesAndChunks = f.queries](lsst::qserv::util::CmdData*) { + auto fastFunc = [&running, &task, queriesAndChunks = fixt.queries](lsst::qserv::util::CmdData*) { queriesAndChunks->startedTask(task); std::this_thread::sleep_for(std::chrono::milliseconds(1)); queriesAndChunks->finishedTask(task); running = true; }; task->setUnitTest(fastFunc); - f.queries->addTask(task); - f.blend->queCmd(task); + fixt.queries->addTask(task); + fixt.blend->queCmd(task); while (!running) { std::this_thread::sleep_for(std::chrono::milliseconds(10)); } running = false; - // f.queries should now have a baseline for chunk 27. - LOGS(_log, LOG_LVL_DEBUG, "Chunks after fastFunc " << *f.queries); + // fixt.queries should now have a baseline for chunk 27. + LOGS(_log, LOG_LVL_DEBUG, "Chunks after fastFunc " << *fixt.queries); - taskMsg = newTaskMsgScan(27, lsst::qserv::proto::ScanInfo::Rating::FAST, qid, 0); - task = makeTask(taskMsg, f.queries); + ujd = makeUberJobData(qidB, scanInfoFast, scanInteractiveF); + task = makeUTask(27, 0, ujd, sc, fixt.queries); std::atomic slowSleepDone{false}; auto slowFunc = [&running, &slowSleepDone, &task, - queriesAndChunks = f.queries](lsst::qserv::util::CmdData*) { + queriesAndChunks = fixt.queries](lsst::qserv::util::CmdData*) { queriesAndChunks->startedTask(task); running = true; std::this_thread::sleep_for(std::chrono::seconds(1)); @@ -788,13 +936,13 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryBootTaskTest) { LOGS(_log, LOG_LVL_DEBUG, "slowFunc end"); }; task->setUnitTest(slowFunc); - f.queries->addTask(task); - auto queryStats = f.queries->getStats(qid); + fixt.queries->addTask(task); + auto queryStats = fixt.queries->getStats(qidA); BOOST_CHECK(queryStats != nullptr); if (queryStats != nullptr) { BOOST_CHECK(queryStats->getTasksBooted() == 0); } - f.blend->queCmd(task); + fixt.blend->queCmd(task); // Wait for slowFunc to start running then wait for slowFunc to finish sleeping. while (!running) { std::this_thread::sleep_for(std::chrono::milliseconds(100)); @@ -805,13 +953,13 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryBootTaskTest) { // By now the slowFunc query has taken a second, far longer than the 0.1 seconds it was allowed. // examineAll() should boot the query. - LOGS(_log, LOG_LVL_INFO, "Chunks after slowFunc " << *f.queries); - f.queries->examineAll(); + LOGS(_log, LOG_LVL_INFO, "Chunks after slowFunc " << *fixt.queries); + fixt.queries->examineAll(); running = false; // allow slowFunc to exit its loop and finish. - LOGS(_log, LOG_LVL_INFO, "Chunks after examineAll " << *f.queries); + LOGS(_log, LOG_LVL_INFO, "Chunks after examineAll " << *fixt.queries); // Check if the tasks booted value for qid has gone up. - queryStats = f.queries->getStats(qid); + queryStats = fixt.queries->getStats(qidB); BOOST_CHECK(queryStats != nullptr); if (queryStats != nullptr) { LOGS(_log, LOG_LVL_INFO, "taskBooted=" << queryStats->getTasksBooted()); @@ -821,51 +969,83 @@ BOOST_AUTO_TEST_CASE(BlendScheduleQueryBootTaskTest) { LOGS(_log, LOG_LVL_INFO, "BlendScheduleQueryBootTaskTest waiting for pool to finish."); pool->shutdownPool(); LOGS(_log, LOG_LVL_INFO, "BlendScheduleQueryBootTaskTest done"); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(SlowTableHeapTest) { -#if 0 // &&& fix and re-enable - LOGS(_log, LOG_LVL_DEBUG, "Test_case SlowTableHeapTest start"); - auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, - maxDarkTasksC, resetForTestingC); + LOGS(_log, LOG_LVL_DEBUG, "Test_case SlowTableHeapTest start, see ScanInfo::compareTables"); + + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); // Values to keep QueriesAndChunk from triggering. + + bool const scanInteractiveF = false; + shared_ptr sc = nullptr; wsched::ChunkTasks::SlowTableHeap heap{}; lsst::qserv::QueryId qIdInc = 1; BOOST_CHECK(heap.empty() == true); - Task::Ptr a1 = makeTask(newTaskMsgScan(7, 3, qIdInc++, 0, "charlie"), queries); + auto scanI = makeScanInfoMedium("charlie"); + auto ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a1 = makeUTask(7, 0, ujd, sc, fixt.queries); heap.push(a1); BOOST_CHECK(heap.top().get() == a1.get()); BOOST_CHECK(heap.empty() == false); - Task::Ptr a2 = makeTask(newTaskMsgScan(7, 3, qIdInc++, 0, "delta"), queries); + scanI = makeScanInfoMedium("delta"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a2 = makeUTask(7, 0, ujd, sc, fixt.queries); heap.push(a2); - BOOST_CHECK(heap.top().get() == a2.get()); + auto hTop = heap.top(); + logCmd(hTop, "hTop a2"); + BOOST_CHECK(hTop.get() == a2.get()); - Task::Ptr a3 = makeTask(newTaskMsgScan(7, 4, qIdInc++, 0, "bravo"), queries); + scanI = makeScanInfoSlow("bravo"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a3 = makeUTask(7, 0, ujd, sc, fixt.queries); heap.push(a3); + hTop = heap.top(); + logCmd(hTop, "hTop a3 first"); BOOST_CHECK(heap.top().get() == a3.get()); - Task::Ptr a4 = makeTask(newTaskMsgScan(7, 2, qIdInc++, 0, "alpha"), queries); + scanI = makeScanInfoFast("alpha"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a4 = makeUTask(7, 0, ujd, sc, fixt.queries); heap.push(a4); + hTop = heap.top(); + logCmd(hTop, "hTop a3 second"); BOOST_CHECK(heap.top().get() == a3.get()); BOOST_CHECK(heap.size() == 4); - BOOST_CHECK(heap.pop().get() == a3.get()); - BOOST_CHECK(heap.pop().get() == a2.get()); - BOOST_CHECK(heap.pop().get() == a1.get()); - BOOST_CHECK(heap.pop().get() == a4.get()); + auto hPop = heap.pop(); + logCmd(hPop, "hPop expect a3 bravo"); + BOOST_CHECK(hPop.get() == a3.get()); + + hPop = heap.pop(); + logCmd(hPop, "hPop expect a2 delta"); + BOOST_CHECK(hPop.get() == a2.get()); + + hPop = heap.pop(); + logCmd(hPop, "hPop expect a1 charlie"); + BOOST_CHECK(hPop.get() == a1.get()); + + hPop = heap.pop(); + logCmd(hPop, "hPop expect a4 alpha"); + BOOST_CHECK(hPop.get() == a4.get()); BOOST_CHECK(heap.empty() == true); LOGS(_log, LOG_LVL_DEBUG, "SlowTableHeapTest done"); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(ChunkTasksTest) { -#if 0 // &&& fix and re-enable - LOGS(_log, LOG_LVL_DEBUG, "Test_case ChunkTasksTest start"); - auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, - maxDarkTasksC, resetForTestingC); + LOGS(_log, LOG_LVL_DEBUG, "Test_case ChunkTasksTest start, see ScanInfo::compareTables"); + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); + shared_ptr sc = nullptr; + bool const scanInteractiveF = false; + int chunkId = 7; wsched::ChunkTasks chunkTasks{chunkId}; lsst::qserv::QueryId qIdInc = 1; @@ -873,21 +1053,29 @@ BOOST_AUTO_TEST_CASE(ChunkTasksTest) { BOOST_CHECK(chunkTasks.empty() == true); BOOST_CHECK(chunkTasks.readyToAdvance() == true); - Task::Ptr a1 = makeTask(newTaskMsgScan(chunkId, 3, qIdInc++, 0, "charlie"), queries); + auto scanI = makeScanInfoMedium("charlie"); + auto ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a1 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); chunkTasks.queTask(a1); BOOST_CHECK(chunkTasks.empty() == false); BOOST_CHECK(chunkTasks.readyToAdvance() == false); BOOST_CHECK(chunkTasks.size() == 1); - Task::Ptr a2 = makeTask(newTaskMsgScan(chunkId, 3, qIdInc++, 0, "delta"), queries); + scanI = makeScanInfoMedium("delta"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a2 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); chunkTasks.queTask(a2); BOOST_CHECK(chunkTasks.size() == 2); - Task::Ptr a3 = makeTask(newTaskMsgScan(chunkId, 4, qIdInc++, 0, "bravo"), queries); + scanI = makeScanInfoSlow("bravo"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a3 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); chunkTasks.queTask(a3); BOOST_CHECK(chunkTasks.size() == 3); - Task::Ptr a4 = makeTask(newTaskMsgScan(chunkId, 2, qIdInc++, 0, "alpha"), queries); + scanI = makeScanInfoFast("alpha"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a4 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); chunkTasks.queTask(a4); BOOST_CHECK(chunkTasks.size() == 4); @@ -931,14 +1119,18 @@ BOOST_AUTO_TEST_CASE(ChunkTasksTest) { chunkTasks.taskComplete(a4); BOOST_CHECK(chunkTasks.readyToAdvance() == true); LOGS(_log, LOG_LVL_DEBUG, "ChunkTasksTest done"); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_CASE(ChunkTasksQueueTest) { -#if 0 // &&& fix and re-enable - LOGS(_log, LOG_LVL_DEBUG, "Test_case ChunkTasksQueueTest start"); - auto queries = QueriesAndChunks::setupGlobal(chrono::seconds(1), chrono::seconds(300), maxBootedC, - maxDarkTasksC, resetForTestingC); + LOGS(_log, LOG_LVL_DEBUG, "Test_case ChunkTasksQueueTest start, see ScanInfo::compareTables"); + + int const deadAfter = 1; + int const examineAfter = 1; + auto qac = setupQueries(maxBootedC, maxDarkTasksC, resetForTestingC, deadAfter, examineAfter); + SchedFixture fixt(60.0, qac); + shared_ptr sc = nullptr; + bool const scanInteractiveF = false; + int firstChunkId = 100; int secondChunkId = 150; int chunkId = firstChunkId; @@ -948,15 +1140,25 @@ BOOST_AUTO_TEST_CASE(ChunkTasksQueueTest) { BOOST_CHECK(ctl.empty() == true); BOOST_CHECK(ctl.ready(true) == false); - Task::Ptr a1 = makeTask(newTaskMsgScan(chunkId, 3, qIdInc++, 0, "charlie"), queries); + auto scanI = makeScanInfoMedium("charlie"); + auto ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a1 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(a1); BOOST_CHECK(ctl.empty() == false); - Task::Ptr a2 = makeTask(newTaskMsgScan(chunkId, 3, qIdInc++, 0, "delta"), queries); + scanI = makeScanInfoMedium("delta"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a2 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(a2); - Task::Ptr a3 = makeTask(newTaskMsgScan(chunkId, 4, qIdInc++, 0, "bravo"), queries); + + scanI = makeScanInfoSlow("bravo"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a3 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(a3); - Task::Ptr a4 = makeTask(newTaskMsgScan(chunkId, 2, qIdInc++, 0, "alpha"), queries); + + scanI = makeScanInfoFast("alpha"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr a4 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(a4); BOOST_CHECK(ctl.ready(true) == true); @@ -974,15 +1176,25 @@ BOOST_AUTO_TEST_CASE(ChunkTasksQueueTest) { BOOST_CHECK(ctl.empty() == true); chunkId = secondChunkId; - Task::Ptr b1 = makeTask(newTaskMsgScan(chunkId, 3, qIdInc++, 0, "c"), queries); + scanI = makeScanInfoMedium("c"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr b1 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(b1); BOOST_CHECK(ctl.empty() == false); - Task::Ptr b2 = makeTask(newTaskMsgScan(chunkId, 3, qIdInc++, 0, "d"), queries); + scanI = makeScanInfoMedium("d"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr b2 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(b2); - Task::Ptr b3 = makeTask(newTaskMsgScan(chunkId, 4, qIdInc++, 0, "b"), queries); + + scanI = makeScanInfoSlow("b"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr b3 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(b3); - Task::Ptr b4 = makeTask(newTaskMsgScan(chunkId, 2, qIdInc++, 0, "a"), queries); + + scanI = makeScanInfoFast("a"); + ujd = makeUberJobData(qIdInc++, scanI, scanInteractiveF); + Task::Ptr b4 = makeUTask(chunkId, 0, ujd, sc, fixt.queries); ctl.queueTask(b4); ctl.queueTask(a3); ctl.queueTask(a4); @@ -1049,7 +1261,6 @@ BOOST_AUTO_TEST_CASE(ChunkTasksQueueTest) { BOOST_CHECK(ctl.ready(true) == false); BOOST_CHECK(ctl.getActiveChunkId() == -1); LOGS(_log, LOG_LVL_DEBUG, "ChunkTasksQueueTest done"); -#endif // &&& fix and re-enable } BOOST_AUTO_TEST_SUITE_END()