From 5930ae6200b938ac26e38a3a89cad1c923d4ae54 Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 25 Apr 2024 17:15:45 -0700 Subject: [PATCH 01/15] Added UberJob branch code. --- .../lsst/qserv/admin/qservCli/launch.py | 2 + src/ccontrol/MergingHandler.cc | 9 + src/ccontrol/UserQueryFactory.cc | 1 + src/ccontrol/UserQuerySelect.cc | 337 +++++++++++++++++- src/ccontrol/UserQuerySelect.h | 1 + src/czar/Czar.cc | 22 +- src/czar/Czar.h | 1 + src/czar/CzarChunkMap.cc | 20 +- src/czar/CzarChunkMap.h | 17 +- src/czar/CzarRegistry.cc | 20 +- src/czar/CzarRegistry.h | 11 +- src/czar/testCzar.cc | 25 +- src/proto/worker.proto | 12 + src/qdisp/CMakeLists.txt | 2 + src/qdisp/Executive.cc | 88 ++++- src/qdisp/Executive.h | 29 +- src/qdisp/JobDescription.cc | 8 + src/qdisp/JobDescription.h | 8 +- src/qdisp/JobQuery.cc | 45 ++- src/qdisp/JobQuery.h | 53 ++- src/qdisp/QueryRequest.cc | 133 ++++++- src/qdisp/QueryRequest.h | 23 +- src/qdisp/ResponseHandler.h | 11 +- src/qdisp/UberJob.cc | 214 +++++++++++ src/qdisp/UberJob.h | 144 ++++++++ src/qmeta/CMakeLists.txt | 16 + src/qmeta/QMeta.h | 51 ++- src/qmeta/QMetaMysql.cc | 8 +- src/qmeta/QMetaMysql.h | 4 +- src/qmeta/testQMeta.cc | 6 +- src/qproc/TaskMsgFactory.cc | 65 ++++ src/qproc/TaskMsgFactory.h | 6 + src/xrdsvc/SsiRequest.cc | 4 + src/xrdsvc/SsiService.cc | 2 + 34 files changed, 1313 insertions(+), 85 deletions(-) create mode 100644 src/qdisp/UberJob.cc create mode 100644 src/qdisp/UberJob.h diff --git a/src/admin/python/lsst/qserv/admin/qservCli/launch.py b/src/admin/python/lsst/qserv/admin/qservCli/launch.py index ebc24cd170..c04babdbdc 100644 --- a/src/admin/python/lsst/qserv/admin/qservCli/launch.py +++ b/src/admin/python/lsst/qserv/admin/qservCli/launch.py @@ -262,7 +262,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 7b78aed245..f2f690f2be 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -403,11 +403,20 @@ bool MergingHandler::flush(proto::ResponseSummary const& responseSummary, uint32 // This is needed to ensure the job query would be staying alive for the duration // of the operation to prevent inconsistency witin the application. + /* &&& auto const jobQuery = getJobQuery().lock(); if (jobQuery == nullptr) { LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobQuery was NULL"); return false; } + */ + auto const jobBase = getJobBase().lock(); + if (jobBase == nullptr) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobBase was NULL"); + return false; + } + auto const jobQuery = std::dynamic_pointer_cast(jobBase); + LOGS(_log, LOG_LVL_TRACE, "MergingHandler::" << __func__ << " jobid=" << responseSummary.jobid() << " transmitsize=" << responseSummary.transmitsize() diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 44aeda0c1f..bfa5f39908 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -226,6 +226,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st std::string query = aQuery; // TODO: DM-43386 need to have WorkerChunkMap info at this point + // &&& std::string stripped; bool async = false; diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 91a7b21ae9..0486e2711b 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -80,12 +80,16 @@ #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 "qmeta/QMeta.h" #include "qmeta/Exceptions.h" #include "qproc/geomAdapter.h" @@ -102,14 +106,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 @@ -233,7 +241,7 @@ std::string UserQuerySelect::getResultQuery() const { } /// Begin running on all chunks added so far. -void UserQuerySelect::submit() { +void UserQuerySelect::submitOld() { // &&& _qSession->finalize(); // Using the QuerySession, generate query specs (text, db, chunkId) and then @@ -271,6 +279,7 @@ void UserQuerySelect::submit() { _executive->setScanInteractive(_qSession->getScanInteractive()); + // &&& for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); ++i) { auto& chunkSpec = *i; @@ -318,6 +327,332 @@ void UserQuerySelect::submit() { } } +/// Begin running on all chunks added so far. +void UserQuerySelect::submit() { //&&&uj + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew start"); + _qSession->finalize(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew a"); + + // Using the QuerySession, generate query specs (text, db, chunkId) and then + // create query messages and send them to the async query manager. + LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect beginning submission"); + assert(_infileMerger); + + //&&&auto taskMsgFactory = std::make_shared(_qMetaQueryId); + auto taskMsgFactory = std::make_shared(); + TmpTableName ttn(_qMetaQueryId, _qSession->getOriginal()); + std::vector chunks; + std::mutex chunksMtx; + int sequence = 0; + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew b"); + 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); + } + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew c"); + // Add QStatsTmp table entry + try { + _queryStatsData->queryStatsTmpRegister(_qMetaQueryId, _qSession->getChunksSize()); + } catch (qmeta::SqlError const& e) { + LOGS(_log, LOG_LVL_WARN, "Failed queryStatsTmpRegister " << e.what()); + } + + _executive->setScanInteractive(_qSession->getScanInteractive()); + + string dbName(""); // it isn't easy to set this //&&&diff + bool dbNameSet = false; //&&&diff + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d"); + for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); + ++i) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d1"); + auto& chunkSpec = *i; + + // Make the JobQuery now + QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); + + qproc::ChunkQuerySpec::Ptr cs; //&&&diff old one did this in lambda + { + std::lock_guard lock(chunksMtx); + cs = _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec); + chunks.push_back(cs->chunkId); + } + std::string chunkResultName = ttn.make(cs->chunkId); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d2"); + // This should only need to be set once as all jobs should have the same database name. + //&&& this probably has to do with locating xrootd resources, need to check. ??? + if (cs->db != dbName) { + LOGS(_log, LOG_LVL_WARN, "&&& dbName change from " << dbName << " to " << cs->db); + if (dbNameSet) { + throw util::Bug(ERR_LOC, "Multiple database names in UBerJob"); + } + dbName = cs->db; + dbNameSet = true; + } + + //&&& TODO:UJ for UberJobs, cmr and MerginHandler wont be needed unless the uber job fails. could + //probably save some time. + //&&& std::shared_ptr cmr = ChunkMsgReceiver::newInstance(cs->chunkId, + //_messageStore); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d3"); + ResourceUnit ru; + ru.setAsDbChunk(cs->db, cs->chunkId); + qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( + _qMetaCzarId, _executive->getId(), sequence, ru, + //&&&std::make_shared(cmr, _infileMerger, chunkResultName), + std::make_shared(_infileMerger, chunkResultName), taskMsgFactory, cs, + chunkResultName); + auto job = _executive->add(jobDesc); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4"); + if (!uberJobsEnabled) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4a"); + // references in captures cause races + auto funcBuildJob = [this, job{move(job)}](util::CmdData*) { + QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); + //&&& job->runJob(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew lambda a"); + _executive->runJobQuery(job); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew lambda b"); + }; + auto cmd = std::make_shared(funcBuildJob); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4b"); + _executive->queueJobStart(cmd); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4c"); + } + ++sequence; + } + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e"); + if (uberJobsEnabled) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); + vector uberJobs; + + auto czarPtr = czar::Czar::getCzar(); + // auto workerResources = czarPtr->getWorkerResourceLists(); //&&& replace with CzarRegistry stuff + auto czChunkMap = czarPtr->getCzarChunkMap(); + auto czRegistry = czarPtr->getCzarRegistry(); + + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj + + // Make a map of all jobs in the executive. + // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can + // be found for all databases in the query + qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); + + // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use + // the same databases) Use this to check for conflicts + + // assign jobs to uberJobs + int maxChunksPerUber = 1; // &&& maybe put in config??? + // keep cycling through workers until no more chunks to place. + + /// make a map that will be destroyed as chunks are checked/used + //&&&uj REPLACE map> tmpWorkerList = workerResources->getDequesFor(dbName); + + // TODO:UJ &&&uj So UberJobIds don't conflict with chunk numbers or jobIds, start at a large number. + // This could use some refinement. + int uberJobId = qdisp::UberJob::getFirstIdNumber(); + +#if 0 // &&&uj + // Keep making UberJobs until either all chunks in the query have been assigned or there are no more + // chunks on workers. + for(auto&& workerIter = tmpWorkerList.begin(); !(chunksInQuery.empty() || tmpWorkerList.empty());) { + /// TODO:UJ One issue here that shouldn't be in a problem in the final version, there are 3 replicas here. + /// The final version should only have LeadChunks, so there shuoldn't be any duplicates. For every hit + /// on the worker, there will be 2 misses. That will probably hurt significantly. + + ///&&& TODO:UJ cs->chunkId in cmr, replacing with uberJobId for now + ///&&& TODO:UJ MergingHandler result name looks like it is only used for log messages. + string uberResultName = "uber_" + to_string(uberJobId); + std::shared_ptr cmr = ChunkMsgReceiver::newInstance(uberJobId, _messageStore); + auto respHandler = std::make_shared(cmr, _infileMerger, uberResultName); + + string workerResourceName = workerIter->first; + deque& dq = workerIter->second; + auto uJob = qdisp::UberJob::create(_executive, respHandler, _qMetaQueryId, + uberJobId++, _qMetaCzarId, workerResourceName); + + int chunksInUber = 0; + while (!dq.empty() && !chunksInQuery.empty() && chunksInUber < maxChunksPerUber) { + int chunkIdWorker = dq.front(); + dq.pop_front(); + auto found = chunksInQuery.find(chunkIdWorker); + if (found != chunksInQuery.end()) { + uJob->addJob(found->second); + ++chunksInUber; + chunksInQuery.erase(found); + } + } + + //LOGS(_log, LOG_LVL_INFO, "&&& making UberJob " << uberResultName << " chunks=" << chunksInUber); + if (chunksInUber > 0) { + uberJobs.push_back(uJob); + } + + // If this worker has no more chunks, remove it from the list. + auto oldWorkerIter = workerIter; + ++workerIter; + if (dq.empty()) { + tmpWorkerList.erase(oldWorkerIter); + } + + // Wrap back to the first worker at the end of the list. + if (workerIter == tmpWorkerList.end()) { + workerIter = tmpWorkerList.begin(); + } + } +#else // &&&uj + // - create a map of UberJobs key=, val=> + // - for chunkId in `chunksInQuery` + // - 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, append a new uberjob to the vect + // - 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. + map> workerJobMap; + vector missingChunks; + + // chunksInQuery 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] : chunksInQuery) { + auto iter = chunkMapPtr->find(chunkId); + if (iter == chunkMapPtr->end()) { + missingChunks.push_back(chunkId); + break; + } + czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; + auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + if (targetWorker == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "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, "Alternate worker found for chunk=" << chunkData->dump()); + } + } + if (!found) { + LOGS(_log, LOG_LVL_ERROR, + "No primary or alternate worker found for chunk=" << chunkData->dump()); + throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + + " Crashing the program here for this reason is not " + "appropriate. &&& NEEDS CODE"); + } + } + // 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() > maxChunksPerUber) { + //&&&shared_ptr cmr = ChunkMsgReceiver::newInstance(uberJobId, + //_messageStore); + 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); + } + ujVect.back()->addJob(jqPtr); + } + + if (!missingChunks.empty()) { + string errStr = string(__func__) + " a worker could not be found for these chunks "; + for (auto const& chk : missingChunks) { + errStr += to_string(chk) + ","; + } + LOGS(_log, LOG_LVL_ERROR, errStr); + throw util::Bug( + ERR_LOC, + errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); + +#endif //&&&uj + +#if 0 //&&&uj + _executive->addUberJobs(uberJobs); + for (auto&& uJob:uberJobs) { + uJob->runUberJob(); + } + LOGS(_log, LOG_LVL_INFO, "&&& All UberJobs sent."); + // If any chunks in the query were not found on a worker's list, run them individually. + for (auto& ciq:chunksInQuery) { + LOGS(_log, LOG_LVL_INFO, "&&& running remaining jobs "); + qdisp::JobQuery* jqRaw = ciq.second; + qdisp::JobQuery::Ptr job = _executive->getSharedPtrForRawJobPtr(jqRaw); + LOGS(_log, LOG_LVL_INFO, "&&& running remaining jobs " << job->getIdStr()); + std::function funcBuildJob = + [this, job{move(job)}](util::CmdData*) { // references in captures cause races + QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); + job->runJob(); + }; + auto cmd = std::make_shared(funcBuildJob); + _executive->queueJobStart(cmd); + } + } +#else //&&&uj + // &&& - How should this work? We have `workerJobMap` which has worker info, but no worker contact info + // Add worker contact info to UberJobs.&&&todo + auto const wContactMap = czRegistry->getWorkerContactMap(); + for (auto const& [wIdKey, ujVect] : workerJobMap) { + auto iter = wContactMap->find(wIdKey); + if (iter == wContactMap->end()) { + // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. + throw util::Bug(ERR_LOC, string(" &&&uj NEED CODE, 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) { + //&&&ujPtr->runUberJob(); + //&&&_executive->runJobQuery(job); + _executive->runUberJob(ujPtr); + } + } +#endif //&&&uj + + // attempt to restore original thread priority, requires root + if (increaseThreadPriority) { + threadPriority.restoreOriginalValues(); + } + + LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); + _executive->waitForAllJobsToStart(); + + // we only care about per-chunk info for ASYNC queries + if (_async) { + std::lock_guard lock(chunksMtx); + _qMetaAddChunks(chunks); + } + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); +} + /// 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 70b7d87a89..b0ffea4e71 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -112,6 +112,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 d7e588c346..862a32cff4 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -76,12 +76,14 @@ 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"); @@ -272,6 +274,11 @@ Czar::Czar(string const& configFilePath, string const& czarName) _czarRegistry = CzarRegistry::create(_czarConfig); } +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)); @@ -516,8 +523,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); @@ -537,7 +551,7 @@ void Czar::removeOldResultTables() { _lastRemovedTimer.start(); _removingOldTables = true; // Run in a separate thread in the off chance this takes a while. - thread t([this]() { + thread thrd([this]() { LOGS(_log, LOG_LVL_INFO, "Removing old result database tables."); auto sqlConn = sql::SqlConnectionFactory::make(_czarConfig->getMySqlResultConfig()); string dbName = _czarConfig->getMySqlResultConfig().dbName; @@ -583,8 +597,8 @@ void Czar::removeOldResultTables() { } _removingOldTables = false; }); - t.detach(); - _oldTableRemovalThread = std::move(t); + thrd.detach(); + _oldTableRemovalThread = std::move(thrd); } SubmitResult Czar::getQueryInfo(QueryId queryId) const { diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 36878c9aa5..f279acd583 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -77,6 +77,7 @@ class Czar { Czar(Czar const&) = delete; Czar& operator=(Czar const&) = delete; + ~Czar(); /** * Submit query for execution. diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 164ad9dd3e..2e81d95e21 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -25,11 +25,13 @@ // System headers #include +#include // &&& del // LSST headers #include "lsst/log/Log.h" // Qserv headers +#include "qmeta/QMeta.h" //&&& move and check linking #include "czar/Czar.h" #include "czar/CzarRegistry.h" #include "qmeta/Exceptions.h" @@ -45,6 +47,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarChunkMap"); namespace lsst::qserv::czar { CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { + cout << "&&& CzarChunkMap::CzarChunkMap()" << endl; try { auto mapsSet = _read(); if (!mapsSet) { @@ -56,12 +59,17 @@ CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta( } } +CzarChunkMap::~CzarChunkMap() { + LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); + cout << "&&& CzarChunkMap::~CzarChunkMap()" << endl; +} + bool CzarChunkMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_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(_mapMtx); - qmeta::QMeta::ChunkMap qChunkMap = _qmeta->getChunkMap(); + qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(); if (_lastUpdateTime >= qChunkMap.updateTime) { LOGS(_log, LOG_LVL_DEBUG, __func__ << " CzarChunkMap no need to read " @@ -87,7 +95,7 @@ bool CzarChunkMap::_read() { } pair, shared_ptr> CzarChunkMap::makeNewMaps( - qmeta::QMeta::ChunkMap const& qChunkMap) { + qmeta::QMetaChunkMap const& qChunkMap) { // Create new maps. auto wcMapPtr = make_shared(); auto chunkMapPtr = make_shared(); @@ -99,7 +107,7 @@ pair, shared_ptr Chunks map for (auto const& [tableName, chunks] : tables) { // vector of ChunkInfo - for (qmeta::QMeta::ChunkMap::ChunkInfo const& chunkInfo : chunks) { + for (qmeta::QMetaChunkMap::ChunkInfo const& chunkInfo : chunks) { try { int64_t chunkNum = chunkInfo.chunk; SizeT sz = chunkInfo.size; @@ -328,6 +336,12 @@ void CzarChunkMap::ChunkData::addToWorkerHasThis(std::shared_ptr_workerId] = worker; } +std::map> +CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() const { + std::map> newMap = _workerHasThisMap; + return newMap; +} + string CzarChunkMap::ChunkData::dump() const { stringstream os; auto primaryWorker = _primaryScanWorker.lock(); diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index f15eda58f8..fe414e62c5 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -26,17 +26,23 @@ // System headers #include #include +#include #include #include -#include +//&&&#include #include #include // Qserv headers #include "global/clock_defs.h" -#include "qmeta/QMeta.h" +// #include "qmeta/QMeta.h" &&& #include "util/Issue.h" +namespace lsst::qserv::qmeta { +class QMeta; +struct QMetaChunkMap; +} // namespace lsst::qserv::qmeta + namespace lsst::qserv::czar { class ChunkMapException : public util::Issue { @@ -84,6 +90,8 @@ class CzarChunkMap { static Ptr create(std::shared_ptr const& qmeta) { return Ptr(new CzarChunkMap(qmeta)); } + ~CzarChunkMap(); + class WorkerChunksData; /// Essentially a structure for storing data about which tables and workers are associated with this @@ -103,6 +111,9 @@ class CzarChunkMap { /// 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; @@ -187,7 +198,7 @@ class CzarChunkMap { /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. static std::pair, std::shared_ptr> - makeNewMaps(qmeta::QMeta::ChunkMap const& qChunkMap); + makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); /// Verify that all chunks belong to at least one worker and that all chunks are represented in shared /// scans. diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index 074ba9bba6..84103631ea 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -49,8 +49,9 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarRegistry"); 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 - // in the detached thread. This will continue before the application gets terminated. + cout << "&&& CzarRegistry::CzarRegistry a" << endl; + // 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); @@ -59,16 +60,22 @@ CzarRegistry::CzarRegistry(std::shared_ptr const& czarConfi } 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; } 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"; @@ -99,9 +106,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 vector const headers; auto const method = http::Method::GET; @@ -135,6 +144,7 @@ void CzarRegistry::_registryWorkerInfoLoop() { } this_thread::sleep_for(chrono::seconds(15)); } + cout << "&&& CzarRegistry::_registryWorkerInfoLoop end" << endl; } CzarRegistry::WorkerContactMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { @@ -149,13 +159,13 @@ CzarRegistry::WorkerContactMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json int wPort = jsQserv.at("management-port").get(); uint64_t updateTimeInt = jsQserv.at("update-time-ms").get(); TIMEPOINT updateTime = TIMEPOINT(chrono::milliseconds(updateTimeInt)); - WorkerContactInfo wInfo(key, wHost, wManagementHost, wPort, updateTime); + 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)) { + if (!wInfo->sameContactInfo(*(iter->second))) { LOGS(_log, LOG_LVL_ERROR, __func__ << " incongruent key " << key << " in " << response); return nullptr; } @@ -180,7 +190,7 @@ bool CzarRegistry::_compareMap(WorkerContactMap const& other) const { if (iter == other.end()) { return false; } else { - if (!(iter->second.sameContactInfo(wInfo))) { + if (!(iter->second->sameContactInfo(*wInfo))) { return false; } } diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index dd51d44098..11c02d57d6 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -67,6 +67,8 @@ class CzarRegistry { ~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_), @@ -87,9 +89,16 @@ class CzarRegistry { } }; - using WorkerContactMap = std::unordered_map; + 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); diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index cde8e59f2b..f8a379f4f5 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -37,8 +37,8 @@ #include "lsst/log/Log.h" // Qserv headers -#include "czar/CzarChunkMap.h" #include "qmeta/QMeta.h" +#include "czar/CzarChunkMap.h" namespace test = boost::test_tools; using namespace lsst::qserv; @@ -51,13 +51,13 @@ using namespace std; BOOST_AUTO_TEST_SUITE(Suite) -void insertIntoQChunkMap(qmeta::QMeta::ChunkMap& qChunkMap, string const& workerId, string const& dbName, +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::QMeta::ChunkMap::ChunkInfo{chunkNum, sz}); + qChunkMap.workers[workerId][dbName][tableName].push_back(qmeta::QMetaChunkMap::ChunkInfo{chunkNum, sz}); } -qmeta::QMeta::ChunkMap convertJsonToChunkMap(nlohmann::json const& jsChunks) { - qmeta::QMeta::ChunkMap qChunkMap; +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()) { @@ -89,6 +89,7 @@ qmeta::QMeta::ChunkMap 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,18 +187,27 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } } )"; + cerr << "&&& c" << endl; auto jsTest1 = nlohmann::json::parse(test1); - qmeta::QMeta::ChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); + cerr << "&&& d" << endl; + qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); + cerr << "&&& e" << endl; auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. + cerr << "&&& f" << endl; LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 1 passed"); + cerr << "&&& g" << endl; auto jsTest2 = nlohmann::json::parse(test2); - qmeta::QMeta::ChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); + cerr << "&&& h" << endl; + qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); + cerr << "&&& i" << endl; tie(chunkMapPtr, wcMapPtr) = czar::CzarChunkMap::makeNewMaps(qChunkMap2); + cerr << "&&& j" << endl; czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 2 passed"); + cerr << "&&& end" << endl; } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/proto/worker.proto b/src/proto/worker.proto index 08f75297ef..418a0e6d5d 100644 --- a/src/proto/worker.proto +++ b/src/proto/worker.proto @@ -75,6 +75,18 @@ message TaskMsg { optional int32 maxtablesize_mb = 15 [default = 0]; } +//&&&uj +// TODO:UJ For now, just a pile of task messages. Eventually, there should be a template +// with chunkId numbers which the worker would then expand into full queries. Much +// like how subchunks are now handled. +message UberJobMsg { + required uint64 queryid = 1; + required uint32 czarid = 2; + required uint32 uberjobid = 3; + repeated TaskMsg taskmsgs = 4; + required uint32 magicnumber = 5; +} + // The file-based result delivery protocol has two kinds of messages. // // 1. The summary message sent back to Czar over the XROOTD/SSI protocol: diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index 567266888c..8d6456e196 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -5,12 +5,14 @@ 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 defcf56541..b2c1e07a30 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -66,6 +66,7 @@ #include "qdisp/MessageStore.h" #include "qdisp/QueryRequest.h" #include "qdisp/ResponseHandler.h" +#include "qdisp/UberJob.h" #include "qdisp/XrdSsiMocks.h" #include "query/QueryContext.h" #include "qproc/QuerySession.h" @@ -174,6 +175,7 @@ void Executive::setQueryId(QueryId id) { /// 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. @@ -210,26 +212,50 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { } QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getIdInt()); - + /* &&&uj + //&&&uj code just returns the jobQuery at this point, it doesn't call runJob(). LOGS(_log, LOG_LVL_DEBUG, "Executive::add with path=" << jobDesc->resource().path()); bool started = jobQuery->runJob(); if (!started && isLimitRowComplete()) { markCompleted(jobQuery->getIdInt(), false); } + */ + LOGS(_log, LOG_LVL_WARN, "&&& Executive::add end"); return jobQuery; } +void Executive::runJobQuery(JobQuery::Ptr const& jobQuery) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::runJobQuery start"); + bool started = jobQuery->runJob(); + if (!started && isLimitRowComplete()) { + markCompleted(jobQuery->getIdInt(), false); + } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::runJobQuery end"); +} + void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueJobStart start"); _jobStartCmdList.push_back(cmd); if (_scanInteractive) { _qdispPool->queCmd(cmd, 0); } 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(); @@ -239,13 +265,16 @@ void Executive::waitForAllJobsToStart() { cmd->waitComplete(); } LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart done"); + LOGS(_log, LOG_LVL_WARN, "&&& waitForAllJobsToStart end"); } // 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) // -bool Executive::startQuery(shared_ptr const& jobQuery) { +bool Executive::startQuery(shared_ptr const& jobQuery) { // &&& + LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery start"); lock_guard lock(_cancelled.getMutex()); + LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery a"); // If we have been cancelled, then return false. // @@ -268,9 +297,64 @@ bool Executive::startQuery(shared_ptr const& jobQuery) { // Start the query. The rest is magically done in the background. // getXrdSsiService()->ProcessRequest(*(qr.get()), jobResource); + LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery end"); return true; } +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 f44abdc17c..cf34e5f51b 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -30,7 +30,6 @@ #include #include #include -#include #include // Third-party headers @@ -51,6 +50,9 @@ #include "util/threadSafe.h" #include "util/ThreadPool.h" +//&&& replace with better enable/disable feature. +#define uberJobsEnabled 0 //&&& + // Forward declarations class XrdSsiService; @@ -67,6 +69,7 @@ class QuerySession; namespace qdisp { class JobQuery; class MessageStore; +class UberJob; //&&&uj } // namespace qdisp namespace util { @@ -92,6 +95,8 @@ class Executive : public std::enable_shared_from_this { public: typedef std::shared_ptr Ptr; typedef std::unordered_map> JobMap; + typedef int ChunkIdType; //&&&uj This type is probably not needed + typedef std::map ChunkIdJobMapType; /// Construct an Executive. /// If c->serviceUrl == ExecutiveConfig::getMockStr(), then use XrdSsiServiceMock @@ -107,6 +112,12 @@ class Executive : public std::enable_shared_from_this { /// Add an item with a reference number std::shared_ptr add(JobDescription::Ptr const& s); + /// &&& doc + void runJobQuery(std::shared_ptr const& jobQuery); + + // &&&uj doc + 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); @@ -167,6 +178,13 @@ 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); /// &&& + + ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& + bool startUberJob(std::shared_ptr const& uJob); /// &&& + std::shared_ptr getSharedPtrForRawJobPtr(JobQuery* jqRaw); /// &&& + private: Executive(ExecutiveConfig const& c, std::shared_ptr const& ms, SharedResources::Ptr const& sharedResources, std::shared_ptr const& qStatus, @@ -240,6 +258,15 @@ class Executive : public std::enable_shared_from_this { bool _scanInteractive = false; ///< true for interactive scans. + // 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 _limitRowComplete{false}; diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 47abd2b4e9..8637d3fe2f 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -35,6 +35,7 @@ // Qserv headers #include "proto/ProtoImporter.h" #include "proto/worker.pb.h" +#include "util/Bug.h" #include "qdisp/ResponseHandler.h" #include "qproc/ChunkQuerySpec.h" #include "qproc/TaskMsgFactory.h" @@ -83,6 +84,13 @@ void JobDescription::buildPayload() { _payloads[_attemptCount] = os.str(); } +bool JobDescription::fillTaskMsg(proto::TaskMsg* tMsg) { //&&&uj + //&&&uj FIXNOW return _taskMsgFactory->fillTaskMsg(tMsg, *_chunkQuerySpec, _chunkResultName, _queryId, + //_jobId, _attemptCount, _czarId); + util::Bug(ERR_LOC, "&&& JobDescription::fillTaskMsg"); + return false; +} + bool JobDescription::verifyPayload() const { proto::ProtoImporter pi; if (!_mock && !pi.messageAcceptable(_payloads.at(_attemptCount))) { diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index f449ede77b..9947099e9a 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -41,11 +41,13 @@ namespace lsst::qserv { -namespace qproc { +namespace proto { +class TaskMsg; +} +namespace qproc { class ChunkQuerySpec; class TaskMsgFactory; - } // namespace qproc namespace qdisp { @@ -87,6 +89,8 @@ class JobDescription { bool incrAttemptCountScrubResults(); bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. + 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 2e73f3649c..98f6c2dd97 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -40,10 +40,13 @@ 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, + JobStatus::Ptr const& jobStatus, shared_ptr const& markCompleteFunc, QueryId qid) : _executive(executive), _jobDescription(jobDescription), @@ -54,31 +57,48 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j _qdispPool = executive->getQdispPool(); 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_DEBUG, "~JobQuery"); } /** 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() { +bool JobQuery::runJob() { // &&& QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); + LOGS(_log, LOG_LVL_WARN, "&&& JobQuery::runJob start"); 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->incrAttemptCountScrubResults(); if (!okCount) { @@ -101,10 +121,12 @@ 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()); + //&&& 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"); + LOGS(_log, LOG_LVL_WARN, "&&& JobQuery::runJob success end"); return true; } _inSsi = false; @@ -119,7 +141,7 @@ bool JobQuery::cancel(bool superfluous) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); 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 +155,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) { @@ -169,8 +191,17 @@ bool JobQuery::isQueryCancelled() { return exec->getCancelled(); } +string const& JobQuery::getPayload() const { return _jobDescription->payload(); } + +void JobQuery::callMarkCompleteFunc(bool success) { _markCompleteFunc->operator()(success); } + +/* &&& std::ostream& operator<<(std::ostream& os, JobQuery const& jq) { return os << "{" << jq.getIdStr() << jq._jobDescription << " " << *jq._jobStatus << "}"; } +*/ +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..a5be0c93ff 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" @@ -45,7 +46,8 @@ 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 { +//&&&class JobQuery : public std::enable_shared_from_this { +class JobQuery : public JobBase { public: typedef std::shared_ptr Ptr; @@ -53,20 +55,26 @@ class JobQuery : public std::enable_shared_from_this { static JobQuery::Ptr create(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, JobStatus::Ptr const& jobStatus, std::shared_ptr const& markCompleteFunc, QueryId qid) { - Ptr jq = std::make_shared(executive, jobDescription, jobStatus, markCompleteFunc, 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; } + /// &&& doc + bool runJob(); + + QueryId getQueryId() const override { return _qid; } + int getIdInt() 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; } + + JobStatus::Ptr getStatus() override { return _jobStatus; } void setQueryRequest(std::shared_ptr const& qr) { std::lock_guard lock(_rmutex); @@ -77,16 +85,19 @@ class JobQuery : public std::enable_shared_from_this { return _queryRequestPtr; } - std::shared_ptr getMarkCompleteFunc() { return _markCompleteFunc; } + void callMarkCompleteFunc(bool success) override; + //&&& std::shared_ptr getMarkCompleteFunc() { return _markCompleteFunc; } bool cancel(bool superfluous = false); - bool isQueryCancelled(); + bool isQueryCancelled() override; Executive::Ptr getExecutive() { 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; + + //&&&friend std::ostream& operator<<(std::ostream& os, JobQuery const& jq); /// Make a copy of the job description. JobQuery::_setup() must be called after creation. /// Do not call this directly, use create. @@ -94,10 +105,20 @@ class JobQuery : public std::enable_shared_from_this { JobStatus::Ptr const& jobStatus, std::shared_ptr const& markCompleteFunc, QueryId qid); - bool isCancelled() { return _cancelled; } + //&&&bool isCancelled() { return _cancelled; } + + /// Set to true if this job is part of an UberJob + void setInUberJob(bool inUberJob) { _inUberJob = inUberJob; }; + + /// @return true if this job is part of an UberJob. + bool inUberJob() const { return _inUberJob; } protected: - void _setup() { _jobDescription->respHandler()->setJobQuery(shared_from_this()); } + //&&& void _setup() { _jobDescription->respHandler()->setJobQuery(shared_from_this()); } + void _setup() { + JobBase::Ptr jbPtr = shared_from_this(); + _jobDescription->respHandler()->setJobQuery(jbPtr); + } int _getRunAttemptsCount() const { std::lock_guard lock(_rmutex); @@ -130,6 +151,10 @@ 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; + + /// True if this job is part of an UberJob. + std::atomic _inUberJob{ + false}; ///< TODO:UJ There are probably several places this should be checked }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/QueryRequest.cc b/src/qdisp/QueryRequest.cc index a8d4e8f2ee..b0a8626314 100644 --- a/src/qdisp/QueryRequest.cc +++ b/src/qdisp/QueryRequest.cc @@ -43,6 +43,7 @@ // Qserv headers #include "czar/Czar.h" #include "qdisp/CzarStats.h" +#include "qdisp/UberJob.h" #include "global/LogContext.h" #include "proto/worker.pb.h" #include "qdisp/JobStatus.h" @@ -60,6 +61,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.QueryRequest"); namespace lsst::qserv::qdisp { +/* &&& QueryRequest::QueryRequest(JobQuery::Ptr const& jobQuery) : _jobQuery(jobQuery), _qid(jobQuery->getQueryId()), @@ -69,6 +71,17 @@ QueryRequest::QueryRequest(JobQuery::Ptr const& jobQuery) QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); LOGS(_log, LOG_LVL_TRACE, "New QueryRequest"); } +*/ + +QueryRequest::QueryRequest(JobBase::Ptr const& job) + : _job(job), + _qid(job->getQueryId()), + _jobid(job->getIdInt()), + _jobIdStr(job->getIdStr()), + _qdispPool(_job->getQdispPool()) { + QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); + LOGS(_log, LOG_LVL_TRACE, "New QueryRequest"); +} QueryRequest::~QueryRequest() { QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); @@ -86,16 +99,19 @@ QueryRequest::~QueryRequest() { char* QueryRequest::GetRequest(int& requestLength) { QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); lock_guard lock(_finishStatusMutex); - auto jq = _jobQuery; + //&&&auto jq = _jobQuery; + auto jq = _job; 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(); + //&&&requestLength = jq->getDescription()->payload().size(); + requestLength = jq->getPayload().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()); + //&&&return const_cast(jq->getDescription()->payload().data()); + return const_cast(jq->getPayload().data()); } // Must not throw exceptions: calling thread cannot trap them. @@ -112,7 +128,8 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co } // Make a copy of the _jobQuery shared_ptr in case _jobQuery gets reset by a call to cancel() - auto jq = _jobQuery; + //&&&auto jq = _jobQuery; + auto jq = _job; { lock_guard lock(_finishStatusMutex); if ((_finishStatus != ACTIVE) || (jq == nullptr)) { @@ -124,7 +141,8 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co ostringstream os; os << _jobIdStr << __func__ << " request failed " << getSsiErr(eInfo, nullptr) << " " << GetEndPoint(); - jq->getDescription()->respHandler()->errorFlush(os.str(), -1); + //&&&jq->getDescription()->respHandler()->errorFlush(os.str(), -1); + jq->getRespHandler()->errorFlush(os.str(), -1); jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_ERROR, "SSI"); _errorFinish(); return true; @@ -165,13 +183,18 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co /// 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) { +//&&&bool QueryRequest::_importResultFile(JobQuery::Ptr const& jq) { +bool QueryRequest::_importResultFile(JobBase::Ptr const& job) { // It's possible jq and _jobQuery differ, so need to use jq. - if (jq->isQueryCancelled()) { + if (job->isQueryCancelled()) { LOGS(_log, LOG_LVL_WARN, "QueryRequest::_processData job was cancelled."); _errorFinish(true); return false; } + auto jq = std::dynamic_pointer_cast(job); + if (jq == nullptr) { + throw util::Bug(ERR_LOC, string(__func__) + " unexpected pointer type for job"); + } auto executive = jq->getExecutive(); if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { if (executive == nullptr || executive->getCancelled()) { @@ -222,7 +245,8 @@ bool QueryRequest::_importResultFile(JobQuery::Ptr const& jq) { /// Process an incoming error. bool QueryRequest::_importError(string const& msg, int code) { - auto jq = _jobQuery; + //&&&auto jq = _jobQuery; + auto jq = _job; { lock_guard lock(_finishStatusMutex); if (_finishStatus != ACTIVE || jq == nullptr) { @@ -230,7 +254,7 @@ bool QueryRequest::_importError(string const& msg, int code) { "QueryRequest::_importError code=" << code << " msg=" << msg << " not passed"); return false; } - jq->getDescription()->respHandler()->errorFlush(msg, code); + jq->getRespHandler()->errorFlush(msg, code); } _errorFinish(); return true; @@ -242,8 +266,10 @@ void QueryRequest::ProcessResponseData(XrdSsiErrInfo const& eInfo, char* buff, i throw util::Bug(ERR_LOC, err); } -void QueryRequest::_flushError(JobQuery::Ptr const& jq) { - ResponseHandler::Error err = jq->getDescription()->respHandler()->getError(); +//&&&void QueryRequest::_flushError(JobQuery::Ptr const& jq) { +void QueryRequest::_flushError(JobBase::Ptr const& jq) { + //&&&ResponseHandler::Error err = jq->getDescription()->respHandler()->getError(); + ResponseHandler::Error err = jq->getRespHandler()->getError(); jq->getStatus()->updateInfo(_jobIdStr, JobStatus::MERGE_ERROR, "MERGE", err.getCode(), err.getMsg(), MSG_ERROR); _errorFinish(true); @@ -262,7 +288,7 @@ bool QueryRequest::cancel() { _retried = true; // Prevent retries. // Only call the following if the job is NOT already done. if (_finishStatus == ACTIVE) { - auto jq = _jobQuery; + auto jq = _job; if (jq != nullptr) jq->getStatus()->updateInfo(_jobIdStr, JobStatus::CANCEL, "CANCEL"); } } @@ -272,7 +298,7 @@ bool QueryRequest::cancel() { /// @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; + auto jq = _job; if (jq == nullptr) { // Need to check if _jobQuery is null due to cancellation. return isQueryRequestCancelled(); @@ -303,10 +329,11 @@ void QueryRequest::cleanup() { // _finishStatusMutex before it is unlocked. // This should reset _jobquery and _keepAlive without risk of either being deleted // before being reset. - shared_ptr jq(move(_jobQuery)); + shared_ptr jq(move(_job)); shared_ptr keep(move(_keepAlive)); } +/* &&& /// 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. @@ -314,7 +341,8 @@ void 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; + auto job = _job; + { // Running _errorFinish more than once could cause errors. lock_guard lock(_finishStatusMutex); @@ -341,7 +369,69 @@ bool QueryRequest::_errorFinish(bool 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. + // so this will keep this alive until cleanup() is done. + 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 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 shouldCancel) { + LOGS(_log, LOG_LVL_DEBUG, "_errorFinish() shouldCancel=" << shouldCancel); + + auto jbase = _job; + JobQuery::Ptr jq = dynamic_pointer_cast(jbase); + if (jq == nullptr) { + //&&& TODO:UJ THIS NEEDS WORK - UberJob failures are different than JobQuery failures. + UberJob::Ptr uberJob = dynamic_pointer_cast(jbase); + if (uberJob != nullptr) { + throw util::Bug(ERR_LOC, "&&&NEED_CODE for _errorFinish to work correctly with UberJob"); + // UberJobs breakup into their JobQueries when they fail and run the jobs directly. + } + return false; + } + + // Normal JobQuery error handling. + { + // 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(shouldCancel=" << shouldCancel << ")"); + bool ok = Finished(shouldCancel); + _finishedCalled = true; + if (!ok) { + LOGS(_log, LOG_LVL_ERROR, "QueryRequest::_errorFinish !ok "); + } else { + LOGS(_log, LOG_LVL_DEBUG, "QueryRequest::_errorFinish ok"); + } + + if (!_retried.exchange(true) && !shouldCancel) { + // 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 until cleanup() is done. LOGS(_log, LOG_LVL_DEBUG, "QueryRequest::_errorFinish retrying"); _keepAlive = jq->getQueryRequest(); // shared pointer to this if (!jq->runJob()) { @@ -384,6 +474,7 @@ void QueryRequest::_finish() { 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) { @@ -392,6 +483,16 @@ void QueryRequest::_callMarkComplete(bool success) { if (jq != nullptr) jq->getMarkCompleteFunc()->operator()(success); } } +*/ + +void QueryRequest::_callMarkComplete(bool success) { + if (!_calledMarkComplete.exchange(true)) { + auto jq = _job; + if (jq != nullptr) { + jq->callMarkCompleteFunc(success); + } + } +} ostream& operator<<(ostream& os, QueryRequest const& qr) { os << "QueryRequest " << qr._jobIdStr; diff --git a/src/qdisp/QueryRequest.h b/src/qdisp/QueryRequest.h index 93eedca0f1..6db583ada5 100644 --- a/src/qdisp/QueryRequest.h +++ b/src/qdisp/QueryRequest.h @@ -83,10 +83,16 @@ class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this Ptr; + /* &&& static Ptr create(std::shared_ptr const& jobQuery) { Ptr newQueryRequest(new QueryRequest(jobQuery)); return newQueryRequest; } + */ + static Ptr create(std::shared_ptr const& jobBase) { + Ptr newQueryRequest(new QueryRequest(jobBase)); + return newQueryRequest; + } virtual ~QueryRequest(); @@ -117,19 +123,30 @@ class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this const& jobQuery); + //&&&QueryRequest(std::shared_ptr const& jobQuery); + QueryRequest(JobBase::Ptr const& job); + /// Inform the Executive that this query completed, and call MarkCompleteFunc only once. + /// This should only be called from _finish() or _errorFinish. void _callMarkComplete(bool success); - bool _importResultFile(JobQuery::Ptr const& jq); + //&&&bool _importResultFile(JobQuery::Ptr const& jq); + bool _importResultFile(JobBase::Ptr const& jq); bool _importError(std::string const& msg, int code); bool _errorFinish(bool stopTrying = false); void _finish(); - void _flushError(JobQuery::Ptr const& jq); + //&&&void _flushError(JobQuery::Ptr const& jq); + void _flushError(JobBase::Ptr const& jq); + /* &&& /// 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; + */ + /// Job information. Not using a weak_ptr as Executive could drop its JobBase::Ptr before we're done with + /// it. A call to cancel() could reset _job early, so copy or protect _job with _finishStatusMutex as + /// needed. If (_finishStatus == ACTIVE) _job should be good. + std::shared_ptr _job; std::atomic _retried{false}; ///< Protect against multiple retries of _jobQuery from a /// single QueryRequest. diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 95a82f8357..27f1fad591 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,8 @@ class ResponseHandler { typedef std::shared_ptr Ptr; ResponseHandler() {} - void setJobQuery(std::shared_ptr const& jobQuery) { _jobQuery = jobQuery; } + //&&&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 @@ -85,10 +86,12 @@ class ResponseHandler { /// Scrub the results from jobId-attempt from the result table. virtual void prepScrubResults(int jobId, int attempt) = 0; - std::weak_ptr getJobQuery() { return _jobQuery; } + //&&& std::weak_ptr getJobQuery() { return _jobQuery; } + std::weak_ptr getJobBase() { return _jobBase; } private: - std::weak_ptr _jobQuery; + //&&& 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..cef081363e --- /dev/null +++ b/src/qdisp/UberJob.cc @@ -0,0 +1,214 @@ +/* + * 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 + +// Qserv headers +#include "global/LogContext.h" +#include "proto/ProtoImporter.h" +#include "proto/worker.pb.h" +#include "qdisp/JobQuery.h" +#include "util/Bug.h" + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; + +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, string const& workerResource) { + UberJob::Ptr uJob(new UberJob(executive, respHandler, queryId, uberJobId, czarId, workerResource)); + uJob->_setup(); + return uJob; +} + +UberJob::UberJob(Executive::Ptr const& executive, + std::shared_ptr const& respHandler, + int queryId, int uberJobId, qmeta::CzarId czarId, string const& workerResource) + : JobBase(), _workerResource(workerResource), _executive(executive), + _respHandler(respHandler), _queryId(queryId), _uberJobId(uberJobId), + _czarId(czarId), _idStr("QID=" + to_string(_queryId) + ":uber=" + to_string(uberJobId)) { + _qdispPool = executive->getQdispPool(); + _jobStatus = make_shared(); +} +*/ +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) + ":uber=" + to_string(uberJobId)), + _workerData(workerData) { + _qdispPool = executive->getQdispPool(); + _jobStatus = make_shared(); +} + +bool UberJob::addJob(JobQuery* job) { + bool success = false; + if (job->inUberJob()) { + throw util::Bug(ERR_LOC, string("job already in UberJob job=") + job->dump() + " uberJob=" + dump()); + } + _jobs.push_back(job); + job->setInUberJob(true); + success = true; + return success; +} + +bool UberJob::runUberJob() { + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); + // Build the uberjob payload. + // TODO:UJ For simplicity in the first pass, just make a TaskMsg for each Job and append it to the + // UberJobMsg. + // This is terribly inefficient and should be replaced by using a template and list of chunks that + // the worker fills in, much like subchunks are done now. + { + google::protobuf::Arena arena; + proto::UberJobMsg* ujMsg = google::protobuf::Arena::CreateMessage(&arena); + ujMsg->set_queryid(getQueryId()); + ujMsg->set_czarid(_czarId); + ujMsg->set_uberjobid(_uberJobId); + ujMsg->set_magicnumber(UberJob::getMagicNumber()); + LOGS(_log, LOG_LVL_INFO, "&&& runUberJob sz=" << _jobs.size()); + for (auto&& job : _jobs) { + proto::TaskMsg* tMsg = ujMsg->add_taskmsgs(); + job->getDescription()->fillTaskMsg(tMsg); + } + ujMsg->SerializeToString(&_payload); + } + + auto executive = _executive.lock(); + if (executive == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "runUberJob failed executive==nullptr"); + return false; + } + bool cancelled = executive->getCancelled(); + bool handlerReset = _respHandler->reset(); + bool started = _started.exchange(true); + if (!cancelled && handlerReset && !started) { + auto criticalErr = [this, &executive](std::string const& msg) { + LOGS(_log, LOG_LVL_ERROR, msg << " " << *this << " Canceling user query!"); + executive->squash(); // This should kill all jobs in this user query. + }; + + LOGS(_log, LOG_LVL_DEBUG, "runUberJob verifying payloads"); + if (!verifyPayload()) { + criticalErr("bad payload"); + return false; + } + + // At this point we are all set to actually run the queries. 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, "runUberJob calls StartQuery()"); + std::shared_ptr uJob(dynamic_pointer_cast(shared_from_this())); + _inSsi = true; + if (executive->startUberJob(uJob)) { + //&&&_jobStatus->updateInfo(_idStr, JobStatus::REQUEST); + _jobStatus->updateInfo(_idStr, JobStatus::REQUEST, "EXEC"); + return true; + } + _inSsi = false; + } + LOGS(_log, LOG_LVL_WARN, + "runUberJob failed. cancelled=" << cancelled << " reset=" << handlerReset << " started=" << started); + return false; +} + +void UberJob::prepScrubResults() { + throw util::Bug( + ERR_LOC, + "&&& If needed, prepScrubResults should call prepScrubResults for all JobQueries in the UberJob"); +} + +bool UberJob::isQueryCancelled() { + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_WARN, "_executive == nullptr"); + return true; // Safer to assume the worst. + } + return exec->getCancelled(); +} + +bool UberJob::verifyPayload() const { + proto::ProtoImporter pi; + if (!pi.messageAcceptable(_payload)) { + LOGS(_log, LOG_LVL_DEBUG, _idStr << " Error serializing UberJobMsg."); + return false; + } + return true; +} + +void UberJob::callMarkCompleteFunc(bool success) { + LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); + if (!success) { + throw util::Bug(ERR_LOC, "&&&NEED_CODE may need code to properly handle failed uberjob"); + } + for (auto&& job : _jobs) { + string idStr = job->getIdStr(); + job->getStatus()->updateInfo(idStr, JobStatus::COMPLETE, "COMPLETE"); + job->callMarkCompleteFunc(success); + } +} + +std::ostream& UberJob::dumpOS(std::ostream& os) const { + os << "(workerResource=" << _workerResource << " jobs sz=" << _jobs.size() << "("; + 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..acc3fbc12f --- /dev/null +++ b/src/qdisp/UberJob.h @@ -0,0 +1,144 @@ +/* + * 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. &&& Make non-nested? +#include "czar/CzarRegistry.h" // Need nested class. &&& Make non-nested? +#include "qdisp/Executive.h" +#include "qdisp/JobBase.h" +//&&&#include "qdisp/JobQuery.h" +//&&&#include "qdisp/QueryRequest.h" + +// This header declarations +namespace lsst { namespace qserv { namespace qdisp { + +class QueryRequest; + +class UberJob : public JobBase { +public: + using Ptr = std::shared_ptr; + + static uint32_t getMagicNumber() { return 93452; } + + /* &&& + static Ptr create(Executive::Ptr const& executive, + std::shared_ptr const& respHandler, + int queryId, int uberJobId, qmeta::CzarId czarId, std::string const& workerResource); + */ + + static Ptr create(Executive::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(){}; + + static int getFirstIdNumber() { return 9'000'000; } + + bool addJob(JobQuery* job); + bool runUberJob(); + + QueryId getQueryId() const override { return _queryId; } // TODO:UJ relocate to JobBase + int getIdInt() const override { return _uberJobId; } + 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; } + 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. + + void setQueryRequest(std::shared_ptr const& qr) override { + std::lock_guard lock(_qrMtx); + _queryRequestPtr = qr; + } + + bool verifyPayload() const; + + std::string getWorkerResource() { return _workerResource; } + int getJobCount() const { return _jobs.size(); } + + /// &&& TODO:UJ may not need, + void prepScrubResults(); + + //&&&uj + void setWorkerContactInfo(czar::CzarRegistry::WorkerContactInfo::Ptr const& wContactInfo) { + _wContactInfo = wContactInfo; + } + + //&&&uj + czar::CzarChunkMap::WorkerChunksData::Ptr getWorkerData() { return _workerData; } + + std::ostream& dumpOS(std::ostream& os) const override; + +private: + /* &&& + UberJob(Executive::Ptr const& executive, + std::shared_ptr const& respHandler, + int queryId, int uberJobId, qmeta::CzarId czarId, std::string const& workerResource); + */ + UberJob(Executive::Ptr const& executive, std::shared_ptr const& respHandler, int queryId, + int uberJobId, qmeta::CzarId czarId, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); + + void _setup() { + JobBase::Ptr jbPtr = shared_from_this(); + _respHandler->setJobQuery(jbPtr); + } + + std::vector _jobs; + std::atomic _started{false}; + bool _inSsi = false; + JobStatus::Ptr _jobStatus; + + std::shared_ptr _queryRequestPtr; + std::mutex _qrMtx; + + std::string const _workerResource; + std::string _payload; ///< XrdSsi message to be sent to the _workerResource. + + std::weak_ptr _executive; + std::shared_ptr _respHandler; + int const _queryId; + int const _uberJobId; + qmeta::CzarId _czarId; + + std::string const _idStr; + std::shared_ptr _qdispPool; + + // &&&uj + czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // &&& check if this is needed + + // &&&uj + czar::CzarRegistry::WorkerContactInfo::Ptr _wContactInfo; +}; + +}}} // namespace lsst::qserv::qdisp + +#endif // LSST_QSERV_QDISP_UBERJOB_H diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index faea86ad69..2fbb078f56 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -14,7 +14,23 @@ target_link_libraries(qserv_meta PUBLIC qserv_common log mysqlclient_r + +# ccontrol +# czar +# parser +# rproc +# qana +# qproc +# query ) +# &&& see if there's a way to remove some target_link_libraries +# &&& ccontrol +# &&& czar +# &&& parser +# &&& rproc +# &&& qana +# &&& qproc +# &&& query install(TARGETS qserv_meta) install(DIRECTORY schema/ DESTINATION ${CMAKE_INSTALL_PREFIX}/qserv/smig/qmeta/schema/) diff --git a/src/qmeta/QMeta.h b/src/qmeta/QMeta.h index a8c3a6672a..20af80bdf6 100644 --- a/src/qmeta/QMeta.h +++ b/src/qmeta/QMeta.h @@ -46,6 +46,48 @@ namespace lsst::qserv::qmeta { /// @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. @@ -59,6 +101,7 @@ 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. @@ -76,7 +119,7 @@ class QMeta { * } * @endcode */ - struct ChunkMap { + 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); @@ -100,6 +143,7 @@ class QMeta { /// The last time the map was updated (since UNIX Epoch). TIMEPOINT updateTime; }; +#endif //&&& /** * Create QMeta instance from configuration dictionary. @@ -342,8 +386,9 @@ class QMeta { * @throws EmptyTableError if the corresponding metadata table doesn't have any record * @throws SqlError for any other error related to MySQL */ - virtual ChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = - std::chrono::time_point()) = 0; + 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 40befc5e97..16a4206c78 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -842,10 +842,10 @@ void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr const& prevUpdateTime) { +QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point const& prevUpdateTime) { lock_guard lock(_dbMutex); - QMeta::ChunkMap chunkMap; + QMetaChunkMap chunkMap; auto trans = QMetaTransaction::create(*_conn); @@ -856,7 +856,7 @@ QMeta::ChunkMap QMetaMysql::getChunkMap(chrono::time_point (prevUpdateTime == chrono::time_point()) || (prevUpdateTime < updateTime); if (!force) { trans->commit(); - return QMeta::ChunkMap(); + return QMetaChunkMap(); } // Read the map itself @@ -882,7 +882,7 @@ QMeta::ChunkMap QMetaMysql::getChunkMap(chrono::time_point 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(ChunkMap::ChunkInfo{chunk, size}); + chunkMap.workers[worker][database][table].push_back(QMetaChunkMap::ChunkInfo{chunk, size}); } chunkMap.updateTime = updateTime; } catch (exception const& ex) { diff --git a/src/qmeta/QMetaMysql.h b/src/qmeta/QMetaMysql.h index 34def90969..240d38a066 100644 --- a/src/qmeta/QMetaMysql.h +++ b/src/qmeta/QMetaMysql.h @@ -264,8 +264,8 @@ class QMetaMysql : public QMeta { void addQueryMessages(QueryId queryId, std::shared_ptr const& msgStore) override; /// @see QMeta::getChunkMap - QMeta::ChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = - std::chrono::time_point()) override; + QMetaChunkMap getChunkMap(std::chrono::time_point const& prevUpdateTime = + std::chrono::time_point()) override; protected: /// Check that all necessary tables exist diff --git a/src/qmeta/testQMeta.cc b/src/qmeta/testQMeta.cc index 31cb287b2c..ccb9a47f23 100644 --- a/src/qmeta/testQMeta.cc +++ b/src/qmeta/testQMeta.cc @@ -33,8 +33,8 @@ #include "lsst/log/Log.h" // Qserv headers -#include "QMetaMysql.h" -#include "QStatusMysql.h" +#include "qmeta/QMetaMysql.h" +#include "qmeta/QStatusMysql.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" #include "sql/SqlErrorObject.h" @@ -415,7 +415,7 @@ BOOST_AUTO_TEST_CASE(messWithQueryStats) { BOOST_AUTO_TEST_CASE(getChunkMap) { // The test assumes that the underlying tables exists and it's empty. - QMeta::ChunkMap chunkMap; + QMetaChunkMap chunkMap; BOOST_CHECK_THROW(qMeta->getChunkMap(), EmptyTableError); } diff --git a/src/qproc/TaskMsgFactory.cc b/src/qproc/TaskMsgFactory.cc index ac7a5afe09..e14931a38c 100644 --- a/src/qproc/TaskMsgFactory.cc +++ b/src/qproc/TaskMsgFactory.cc @@ -53,9 +53,73 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.qproc.TaskMsgFactory"); namespace lsst::qserv::qproc { +//&&&uj +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_session(_session); + taskMsg->set_db(chunkQuerySpec.db); + //&&&taskMsg->set_protocol(2); + taskMsg->set_queryid(queryId); + taskMsg->set_jobid(jobId); + taskMsg->set_attemptcount(attemptCount); + taskMsg->set_czarid(czarId); + // LOGS(_log, LOG_LVL_INFO, "&&& _makeMsg ses=" << _session << " db=" << chunkQuerySpec.db << " qId=" << + // queryId << " jId=" << jobId << " att=" << attemptCount << " cz=" << 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) { + LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::_makeMsg start"); std::string resultTable("Asdfasfd"); if (!chunkResultName.empty()) { resultTable = chunkResultName; @@ -110,6 +174,7 @@ std::shared_ptr TaskMsgFactory::_makeMsg(ChunkQuerySpec const& c _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, chunkQuerySpec.queries); } + LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::_makeMsg end"); return taskMsg; } diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h index dc2d0ed130..493e29c605 100644 --- a/src/qproc/TaskMsgFactory.h +++ b/src/qproc/TaskMsgFactory.h @@ -58,6 +58,12 @@ class TaskMsgFactory { virtual void serializeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId, std::ostream& os); + //&&&uj + /// 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); + private: std::shared_ptr _makeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, diff --git a/src/xrdsvc/SsiRequest.cc b/src/xrdsvc/SsiRequest.cc index ec295cfd1f..19db57205c 100644 --- a/src/xrdsvc/SsiRequest.cc +++ b/src/xrdsvc/SsiRequest.cc @@ -90,6 +90,7 @@ 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); @@ -123,6 +124,7 @@ void SsiRequest::execute(XrdSsiRequest& req) { // 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); @@ -186,6 +188,7 @@ void SsiRequest::execute(XrdSsiRequest& req) { 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 { @@ -239,6 +242,7 @@ void SsiRequest::execute(XrdSsiRequest& req) { break; } default: + LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute default"); reportError("Unexpected unit type '" + std::to_string(ru.unitType()) + "', resource name: " + _resourceName); break; diff --git a/src/xrdsvc/SsiService.cc b/src/xrdsvc/SsiService.cc index 473bd5f071..90e770a7ed 100644 --- a/src/xrdsvc/SsiService.cc +++ b/src/xrdsvc/SsiService.cc @@ -283,12 +283,14 @@ SsiService::~SsiService() { void SsiService::ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) { LOGS(_log, LOG_LVL_DEBUG, "Got request call where rName is: " << resRef.rName); + LOGS(_log, LOG_LVL_WARN, "&&& SsiService::ProcessRequest start"); 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); + LOGS(_log, LOG_LVL_WARN, "&&& SsiService::ProcessRequest end"); } } // namespace lsst::qserv::xrdsvc From 6344f31e8741c6e37efe671f28969a8c5251d0b4 Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 16 May 2024 11:22:18 -0700 Subject: [PATCH 02/15] Czar and workers can send http messages to each other. --- src/ccontrol/MergingHandler.cc | 7 - src/ccontrol/UserQueryFactory.cc | 2 +- src/ccontrol/UserQuerySelect.cc | 170 +++++++----------- src/ccontrol/UserQuerySelect.h | 1 - src/czar/CMakeLists.txt | 2 + src/czar/Czar.cc | 14 +- src/czar/CzarChunkMap.cc | 10 +- src/czar/CzarChunkMap.h | 2 - src/czar/CzarRegistry.cc | 10 -- src/czar/HttpCzarWorkerModule.cc | 89 ++++++++++ src/czar/HttpCzarWorkerModule.h | 76 ++++++++ src/czar/HttpSvc.cc | 11 ++ src/czar/testCzar.cc | 11 -- src/http/MetaModule.cc | 2 +- src/http/Module.h | 3 + src/http/RequestBodyJSON.h | 18 ++ src/proto/worker.proto | 2 +- src/qdisp/Executive.cc | 35 ++-- src/qdisp/JobBase.cc | 54 ++++++ src/qdisp/JobBase.h | 70 ++++++++ src/qdisp/JobDescription.cc | 35 +++- src/qdisp/JobDescription.h | 15 +- src/qdisp/JobQuery.cc | 23 --- src/qdisp/JobQuery.h | 8 - src/qdisp/QueryRequest.cc | 74 +------- src/qdisp/ResponseHandler.h | 3 - src/qdisp/UberJob.cc | 112 +++++++++--- src/qdisp/UberJob.h | 27 +-- src/qdisp/testQDisp.cc | 8 + src/qmeta/CMakeLists.txt | 17 +- src/qmeta/QMeta.h | 44 ----- src/qproc/TaskMsgFactory.cc | 125 ++++++++++++- src/qproc/TaskMsgFactory.h | 15 ++ src/xrdsvc/CMakeLists.txt | 1 + src/xrdsvc/HttpReplicaMgtModule.cc | 7 +- src/xrdsvc/HttpSvc.cc | 7 + src/xrdsvc/HttpWorkerCzarModule.cc | 270 +++++++++++++++++++++++++++++ src/xrdsvc/HttpWorkerCzarModule.h | 93 ++++++++++ src/xrdsvc/SsiRequest.cc | 4 - src/xrdsvc/SsiService.cc | 2 - 40 files changed, 1070 insertions(+), 409 deletions(-) create mode 100644 src/czar/HttpCzarWorkerModule.cc create mode 100644 src/czar/HttpCzarWorkerModule.h create mode 100644 src/qdisp/JobBase.cc create mode 100644 src/qdisp/JobBase.h create mode 100644 src/xrdsvc/HttpWorkerCzarModule.cc create mode 100644 src/xrdsvc/HttpWorkerCzarModule.h diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index f2f690f2be..23325b53e5 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -403,13 +403,6 @@ bool MergingHandler::flush(proto::ResponseSummary const& responseSummary, uint32 // This is needed to ensure the job query would be staying alive for the duration // of the operation to prevent inconsistency witin the application. - /* &&& - auto const jobQuery = getJobQuery().lock(); - if (jobQuery == nullptr) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobQuery was NULL"); - return false; - } - */ auto const jobBase = getJobBase().lock(); if (jobBase == nullptr) { LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobBase was NULL"); diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index bfa5f39908..26b808822f 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -226,7 +226,7 @@ UserQuery::Ptr UserQueryFactory::newUserQuery(std::string const& aQuery, std::st std::string query = aQuery; // TODO: DM-43386 need to have WorkerChunkMap info at this point - // &&& + // &&&uj std::string stripped; bool async = false; diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 0486e2711b..8c55b8c7fd 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -240,8 +240,9 @@ std::string UserQuerySelect::getResultQuery() const { return resultQuery; } +#if 0 // &&& /// Begin running on all chunks added so far. -void UserQuerySelect::submitOld() { // &&& +void UserQuerySelect::submitOld() { // &&& to be deleted _qSession->finalize(); // Using the QuerySession, generate query specs (text, db, chunkId) and then @@ -279,7 +280,6 @@ void UserQuerySelect::submitOld() { // &&& _executive->setScanInteractive(_qSession->getScanInteractive()); - // &&& for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); ++i) { auto& chunkSpec = *i; @@ -326,6 +326,7 @@ void UserQuerySelect::submitOld() { // &&& _qMetaAddChunks(chunks); } } +#endif //&&& /// Begin running on all chunks added so far. void UserQuerySelect::submit() { //&&&uj @@ -353,14 +354,6 @@ void UserQuerySelect::submit() { //&&&uj : "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); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew c"); // Add QStatsTmp table entry try { @@ -404,7 +397,7 @@ void UserQuerySelect::submit() { //&&&uj } //&&& TODO:UJ for UberJobs, cmr and MerginHandler wont be needed unless the uber job fails. could - //probably save some time. + // probably save some time. //&&& std::shared_ptr cmr = ChunkMsgReceiver::newInstance(cs->chunkId, //_messageStore); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d3"); @@ -437,7 +430,7 @@ void UserQuerySelect::submit() { //&&&uj } LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e"); - if (uberJobsEnabled) { + if (uberJobsEnabled || true) { LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); vector uberJobs; @@ -446,86 +439,45 @@ void UserQuerySelect::submit() { //&&&uj auto czChunkMap = czarPtr->getCzarChunkMap(); auto czRegistry = czarPtr->getCzarRegistry(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1a"); auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj // Make a map of all jobs in the executive. // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can // be found for all databases in the query qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1b"); // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use // the same databases) Use this to check for conflicts // assign jobs to uberJobs - int maxChunksPerUber = 1; // &&& maybe put in config??? + int maxChunksPerUber = 1; // &&&uj maybe put in config??? or set on command line?? + // &&&uj Different queries may benefit from different values + // &&&uj Such as LIMIT=1 may work best with this at 1, where + // &&&uj 100 would be better for others. // keep cycling through workers until no more chunks to place. - /// make a map that will be destroyed as chunks are checked/used - //&&&uj REPLACE map> tmpWorkerList = workerResources->getDequesFor(dbName); - // TODO:UJ &&&uj So UberJobIds don't conflict with chunk numbers or jobIds, start at a large number. // This could use some refinement. int uberJobId = qdisp::UberJob::getFirstIdNumber(); -#if 0 // &&&uj - // Keep making UberJobs until either all chunks in the query have been assigned or there are no more - // chunks on workers. - for(auto&& workerIter = tmpWorkerList.begin(); !(chunksInQuery.empty() || tmpWorkerList.empty());) { - /// TODO:UJ One issue here that shouldn't be in a problem in the final version, there are 3 replicas here. - /// The final version should only have LeadChunks, so there shuoldn't be any duplicates. For every hit - /// on the worker, there will be 2 misses. That will probably hurt significantly. - - ///&&& TODO:UJ cs->chunkId in cmr, replacing with uberJobId for now - ///&&& TODO:UJ MergingHandler result name looks like it is only used for log messages. - string uberResultName = "uber_" + to_string(uberJobId); - std::shared_ptr cmr = ChunkMsgReceiver::newInstance(uberJobId, _messageStore); - auto respHandler = std::make_shared(cmr, _infileMerger, uberResultName); - - string workerResourceName = workerIter->first; - deque& dq = workerIter->second; - auto uJob = qdisp::UberJob::create(_executive, respHandler, _qMetaQueryId, - uberJobId++, _qMetaCzarId, workerResourceName); - - int chunksInUber = 0; - while (!dq.empty() && !chunksInQuery.empty() && chunksInUber < maxChunksPerUber) { - int chunkIdWorker = dq.front(); - dq.pop_front(); - auto found = chunksInQuery.find(chunkIdWorker); - if (found != chunksInQuery.end()) { - uJob->addJob(found->second); - ++chunksInUber; - chunksInQuery.erase(found); - } - } - - //LOGS(_log, LOG_LVL_INFO, "&&& making UberJob " << uberResultName << " chunks=" << chunksInUber); - if (chunksInUber > 0) { - uberJobs.push_back(uJob); - } - - // If this worker has no more chunks, remove it from the list. - auto oldWorkerIter = workerIter; - ++workerIter; - if (dq.empty()) { - tmpWorkerList.erase(oldWorkerIter); - } - - // Wrap back to the first worker at the end of the list. - if (workerIter == tmpWorkerList.end()) { - workerIter = tmpWorkerList.begin(); - } - } -#else // &&&uj - // - create a map of UberJobs key=, val=> - // - for chunkId in `chunksInQuery` - // - 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, append a new uberjob to the vect - // - 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. + // &&&uj + // - create a map of UberJobs key=, val=> + // - for chunkId in `chunksInQuery` + // - 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, append a new uberjob to the vect + // - 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; @@ -533,14 +485,19 @@ void UserQuerySelect::submit() { //&&&uj // 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. + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c"); for (auto const& [chunkId, jqPtr] : chunksInQuery) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c1"); auto iter = chunkMapPtr->find(chunkId); if (iter == chunkMapPtr->end()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c2"); missingChunks.push_back(chunkId); break; } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c3"); czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c4"); if (targetWorker == nullptr) { LOGS(_log, LOG_LVL_ERROR, "No primary scan worker for chunk=" << chunkData->dump()); // Try to assign a different worker to this job @@ -556,6 +513,11 @@ void UserQuerySelect::submit() { //&&&uj } } if (!found) { + // &&&uj If enough workers are down, there will be a chunk that cannot be found. + // the correct course of action is probably to check the Registry, and + // after so many attempts, cancel the user query with a + // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount + // in the Job or JobDescription could be used for this. LOGS(_log, LOG_LVL_ERROR, "No primary or alternate worker found for chunk=" << chunkData->dump()); throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + @@ -564,21 +526,24 @@ void UserQuerySelect::submit() { //&&&uj } } // 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) { - //&&&shared_ptr cmr = ChunkMsgReceiver::newInstance(uberJobId, - //_messageStore); + 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); } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c7"); if (!missingChunks.empty()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c8"); string errStr = string(__func__) + " a worker could not be found for these chunks "; for (auto const& chk : missingChunks) { errStr += to_string(chk) + ","; @@ -590,34 +555,12 @@ void UserQuerySelect::submit() { //&&&uj } LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); -#endif //&&&uj - -#if 0 //&&&uj - _executive->addUberJobs(uberJobs); - for (auto&& uJob:uberJobs) { - uJob->runUberJob(); - } - LOGS(_log, LOG_LVL_INFO, "&&& All UberJobs sent."); - // If any chunks in the query were not found on a worker's list, run them individually. - for (auto& ciq:chunksInQuery) { - LOGS(_log, LOG_LVL_INFO, "&&& running remaining jobs "); - qdisp::JobQuery* jqRaw = ciq.second; - qdisp::JobQuery::Ptr job = _executive->getSharedPtrForRawJobPtr(jqRaw); - LOGS(_log, LOG_LVL_INFO, "&&& running remaining jobs " << job->getIdStr()); - std::function funcBuildJob = - [this, job{move(job)}](util::CmdData*) { // references in captures cause races - QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); - job->runJob(); - }; - auto cmd = std::make_shared(funcBuildJob); - _executive->queueJobStart(cmd); - } - } -#else //&&&uj - // &&& - How should this work? We have `workerJobMap` which has worker info, but no worker contact info - // Add worker contact info to UberJobs.&&&todo + //&&&uj + // Add worker contact info to UberJobs. auto const wContactMap = czRegistry->getWorkerContactMap(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f"); for (auto const& [wIdKey, ujVect] : workerJobMap) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f1"); auto iter = wContactMap->find(wIdKey); if (iter == wContactMap->end()) { // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. @@ -625,21 +568,26 @@ void UserQuerySelect::submit() { //&&&uj } auto const& wContactInfo = iter->second; for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f2"); ujPtr->setWorkerContactInfo(wContactInfo); } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f3"); _executive->addUberJobs(ujVect); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f4"); for (auto const& ujPtr : ujVect) { - //&&&ujPtr->runUberJob(); - //&&&_executive->runJobQuery(job); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f5"); + //&&&uj IMPORTANT + //&&&uj This just sends the test message to the worker, but the + // worker only parses it and sends a message back. The + // worker does not create and run tasks at this point. + // The call to runUberJob here should be replaced by a call + // to startUberJob that puts the call to runUberJob into + // the a priority queue command. _executive->runUberJob(ujPtr); } } -#endif //&&&uj - // attempt to restore original thread priority, requires root - if (increaseThreadPriority) { - threadPriority.restoreOriginalValues(); - } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); _executive->waitForAllJobsToStart(); @@ -650,7 +598,7 @@ void UserQuerySelect::submit() { //&&&uj _qMetaAddChunks(chunks); } } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew end"); } /// Block until a submit()'ed query completes. diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index b0ffea4e71..70b7d87a89 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -112,7 +112,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/CMakeLists.txt b/src/czar/CMakeLists.txt index 69a3d4e4c1..e10e4bea2b 100644 --- a/src/czar/CMakeLists.txt +++ b/src/czar/CMakeLists.txt @@ -9,6 +9,8 @@ target_sources(czar PRIVATE HttpCzarIngestModule.cc HttpCzarQueryModule.cc HttpCzarSvc.cc + HttpCzarWorkerModule.cc + HttpModule.cc HttpMonitorModule.cc HttpSvc.cc MessageTable.cc diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 862a32cff4..0eaf872abe 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -76,15 +76,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"); /** @@ -274,10 +265,7 @@ Czar::Czar(string const& configFilePath, string const& czarName) _czarRegistry = CzarRegistry::create(_czarConfig); } -Czar::~Czar() { - LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar()"); - cout << "&&& Czar::~Czar()" << endl; -} +Czar::~Czar() { LOGS(_log, LOG_LVL_DEBUG, "Czar::~Czar()"); } 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 2e81d95e21..6f6332fec8 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -25,13 +25,13 @@ // System headers #include -#include // &&& del +//&&& #include // &&& del // LSST headers #include "lsst/log/Log.h" // Qserv headers -#include "qmeta/QMeta.h" //&&& move and check linking +#include "qmeta/QMeta.h" #include "czar/Czar.h" #include "czar/CzarRegistry.h" #include "qmeta/Exceptions.h" @@ -47,7 +47,6 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarChunkMap"); namespace lsst::qserv::czar { CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { - cout << "&&& CzarChunkMap::CzarChunkMap()" << endl; try { auto mapsSet = _read(); if (!mapsSet) { @@ -59,10 +58,7 @@ CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta( } } -CzarChunkMap::~CzarChunkMap() { - LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); - cout << "&&& CzarChunkMap::~CzarChunkMap()" << endl; -} +CzarChunkMap::~CzarChunkMap() { LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); } bool CzarChunkMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_read() start"); diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index fe414e62c5..d4828c0613 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -29,13 +29,11 @@ #include #include #include -//&&&#include #include #include // Qserv headers #include "global/clock_defs.h" -// #include "qmeta/QMeta.h" &&& #include "util/Issue.h" namespace lsst::qserv::qmeta { diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index 84103631ea..368bf66f01 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -49,7 +49,6 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarRegistry"); namespace lsst::qserv::czar { CzarRegistry::CzarRegistry(std::shared_ptr const& czarConfig) : _czarConfig(czarConfig) { - cout << "&&& CzarRegistry::CzarRegistry a" << endl; // 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); @@ -60,22 +59,16 @@ CzarRegistry::CzarRegistry(std::shared_ptr const& czarConfi } 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; } 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,11 +99,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 vector const headers; auto const method = http::Method::GET; @@ -144,7 +135,6 @@ void CzarRegistry::_registryWorkerInfoLoop() { } this_thread::sleep_for(chrono::seconds(15)); } - cout << "&&& CzarRegistry::_registryWorkerInfoLoop end" << endl; } CzarRegistry::WorkerContactMapPtr CzarRegistry::_buildMapFromJson(nlohmann::json const& response) { diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc new file mode 100644 index 0000000000..5d6ac274dc --- /dev/null +++ b/src/czar/HttpCzarWorkerModule.cc @@ -0,0 +1,89 @@ +/* + * 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 "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); + //&&&uj this seems irrelevant for a worker enforceInstanceId(func, + //cconfig::CzarConfig::instance()->replicationInstanceId()); + 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_INFO, __func__ << "&&&uj queryJobError json=" << body().objJson); //&&& + //&&&uj NEED CODE for this + return json::object(); +} + +json HttpCzarWorkerModule::_queryJobReady() { + debug(__func__); + checkApiVersion(__func__, 34); + LOGS(_log, LOG_LVL_INFO, __func__ << "&&&uj queryJobReady json=" << body().objJson); //&&& + //&&&uj NEED CODE for this + json ret = {{"success", 1}}; + return json::object(); +} + +} // namespace lsst::qserv::czar diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h new file mode 100644 index 0000000000..8871162b0c --- /dev/null +++ b/src/czar/HttpCzarWorkerModule.h @@ -0,0 +1,76 @@ +/* + * 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 { + +/// &&& doc This class is used to handle messages to this czar from 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); + + /// &&& doc + nlohmann::json _queryJobError(); + + /// &&& doc + nlohmann::json _queryJobReady(); +}; + +} // 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/testCzar.cc b/src/czar/testCzar.cc index f8a379f4f5..ace8fc86a3 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,27 +185,18 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } } )"; - cerr << "&&& c" << endl; auto jsTest1 = nlohmann::json::parse(test1); - cerr << "&&& d" << endl; qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); - cerr << "&&& e" << endl; auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. - cerr << "&&& f" << endl; LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 1 passed"); - cerr << "&&& g" << endl; auto jsTest2 = nlohmann::json::parse(test2); - cerr << "&&& h" << endl; qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); - cerr << "&&& i" << endl; tie(chunkMapPtr, wcMapPtr) = czar::CzarChunkMap::makeNewMaps(qChunkMap2); - cerr << "&&& j" << endl; czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 2 passed"); - cerr << "&&& end" << endl; } BOOST_AUTO_TEST_SUITE_END() diff --git a/src/http/MetaModule.cc b/src/http/MetaModule.cc index f64572b08d..622d268366 100644 --- a/src/http/MetaModule.cc +++ b/src/http/MetaModule.cc @@ -37,7 +37,7 @@ string const adminAuthKey; namespace lsst::qserv::http { -unsigned int const MetaModule::version = 35; +unsigned int const MetaModule::version = 35; // &&&uj void MetaModule::process(string const& context, nlohmann::json const& info, shared_ptr const& req, shared_ptr const& resp, diff --git a/src/http/Module.h b/src/http/Module.h index 009d2a1923..2864496bf1 100644 --- a/src/http/Module.h +++ b/src/http/Module.h @@ -208,6 +208,9 @@ class Module { */ 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 896250d32e..f82c78a288 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; + + /// &&& doc + /// &&&uj This would be much more efficient if this class had objJson defined as + /// &&&uj a const reference or pointer to const, but implementation 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/worker.proto b/src/proto/worker.proto index 418a0e6d5d..d82af83658 100644 --- a/src/proto/worker.proto +++ b/src/proto/worker.proto @@ -75,7 +75,7 @@ message TaskMsg { optional int32 maxtablesize_mb = 15 [default = 0]; } -//&&&uj +//&&&uj To be deleted. Protobuffs will no longer be needed once xrootd is removed. // TODO:UJ For now, just a pile of task messages. Eventually, there should be a template // with chunkId numbers which the worker would then expand into full queries. Much // like how subchunks are now handled. diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index b2c1e07a30..94c694373d 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -203,6 +203,8 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { LOGS(_log, LOG_LVL_ERROR, "Executive ignoring duplicate track add"); return jobQuery; } + + _addToChunkJobMap(jobQuery); // &&&uj } if (_empty.exchange(false)) { @@ -247,9 +249,11 @@ void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { void Executive::runUberJob(std::shared_ptr const& uberJob) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); bool started = uberJob->runUberJob(); + /* &&&uj if (!started && isLimitRowComplete()) { uberJob->callMarkCompleteFunc(false); } + */ LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob end"); } @@ -296,6 +300,7 @@ bool Executive::startQuery(shared_ptr const& jobQuery) { // &&& // Start the query. The rest is magically done in the background. // + // &&&uj sending mechanism needs to change to be like AddReplicaQservMgtRequest::createHttpReqImp getXrdSsiService()->ProcessRequest(*(qr.get()), jobResource); LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery end"); return true; @@ -324,22 +329,8 @@ bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& // 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); + // &&&uj NEED CODE to put call to runUberJob into the priority queue. - // Start the query. The rest is magically done in the background. - // - getXrdSsiService()->ProcessRequest(*(qr.get()), uJobResource); return true; } @@ -702,6 +693,20 @@ void Executive::_waitAllUntilEmpty() { } } +void Executive::_addToChunkJobMap(JobQuery::Ptr const& job) { + int chunkId = job->getDescription()->resource().chunk(); + auto entry = pair(chunkId, job.get()); + // LOGS(_log, LOG_LVL_WARN, "&&& _addToChunkJobMap chunkId=" << chunkId); + lock_guard lck(_chunkToJobMapMtx); + if (_chunkToJobMapInvalid) { + throw util::Bug(ERR_LOC, "&&& map insert FAILED, map is already invalid"); + } + 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(); 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..787975dc95 --- /dev/null +++ b/src/qdisp/JobBase.h @@ -0,0 +1,70 @@ +/* + * 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" + +// This header declarations +namespace lsst { namespace qserv { namespace qdisp { + +class JobStatus; +class QdispPool; +class ResponseHandler; +class QueryRequest; + +/// Base class for JobQuery and UberJob. +/// TODO:UJ This could use a lot of cleanup. +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 int getIdInt() 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::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 8637d3fe2f..4c6741fdb6 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -77,6 +77,35 @@ bool JobDescription::incrAttemptCountScrubResults() { return true; } +bool JobDescription::incrAttemptCountScrubResultsJson() { +#if 0 //&&&uj this block needs to be reenabled but attempts need to be handled differently ??? + //&&&uj attempt failures generally result from communictaion problems. SQL errors kill the query. + //&&&uj so lots of failed attempts indicate that qserv is unstable. + if (_attemptCount >= 0) { + _respHandler->prepScrubResults(_jobId, _attemptCount); // + } + ++_attemptCount; + if (_attemptCount > MAX_JOB_ATTEMPTS) { + LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); + return false; + } +#endif // &&& + + ++_attemptCount; + if (_attemptCount > MAX_JOB_ATTEMPTS) { + LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); + return false; + } + // build the request + //_payloads[_attemptCount] = os.str(); + auto js = _taskMsgFactory->makeMsgJson(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, + _attemptCount, _czarId); + LOGS(_log, LOG_LVL_ERROR, "&&& JobDescription::incrAttemptCountScrubResultsJson js=" << (*js)); + _jsForWorker = js; + + return true; +} + void JobDescription::buildPayload() { ostringstream os; _taskMsgFactory->serializeMsg(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, _attemptCount, @@ -84,14 +113,14 @@ void JobDescription::buildPayload() { _payloads[_attemptCount] = os.str(); } -bool JobDescription::fillTaskMsg(proto::TaskMsg* tMsg) { //&&&uj - //&&&uj FIXNOW return _taskMsgFactory->fillTaskMsg(tMsg, *_chunkQuerySpec, _chunkResultName, _queryId, +bool JobDescription::fillTaskMsg(proto::TaskMsg* tMsg) { //&&&uj -probably just delete. + //&&&uj return _taskMsgFactory->fillTaskMsg(tMsg, *_chunkQuerySpec, _chunkResultName, _queryId, //_jobId, _attemptCount, _czarId); util::Bug(ERR_LOC, "&&& JobDescription::fillTaskMsg"); return false; } -bool JobDescription::verifyPayload() const { +bool JobDescription::verifyPayload() const { //&&&uj - is there any value to this now? 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 9947099e9a..2e40de8120 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" @@ -86,11 +89,16 @@ class JobDescription { /// @returns true when _attemptCount is incremented correctly and the payload is built. /// If the starting value of _attemptCount was greater than or equal to zero, that /// attempt is scrubbed from the result table. - bool incrAttemptCountScrubResults(); - bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. + bool incrAttemptCountScrubResults(); // &&&uj - to be deleted + bool incrAttemptCountScrubResultsJson(); // &&&uj - scrubbing results probably unneeded with uj. + bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj + std::shared_ptr getJsForWorker() { return _jsForWorker; } + + void resetJsForWorker() { _jsForWorker.reset(); } // &&&uj may need mutex for _jsForWorker + friend std::ostream& operator<<(std::ostream& os, JobDescription const& jd); private: @@ -118,6 +126,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 98f6c2dd97..3be21911c6 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -44,21 +44,6 @@ using namespace std; namespace lsst::qserv::qdisp { -/* &&& -JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, - JobStatus::Ptr const& jobStatus, shared_ptr const& markCompleteFunc, - QueryId qid) - : _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(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, JobStatus::Ptr const& jobStatus, shared_ptr const& markCompleteFunc, QueryId qid) @@ -80,7 +65,6 @@ JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_DEBUG, "~JobQuery"); } */ bool JobQuery::runJob() { // &&& QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); - LOGS(_log, LOG_LVL_WARN, "&&& JobQuery::runJob start"); LOGS(_log, LOG_LVL_DEBUG, " runJob " << *this); auto executive = _executive.lock(); if (executive == nullptr) { @@ -121,12 +105,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"); - LOGS(_log, LOG_LVL_WARN, "&&& JobQuery::runJob success end"); return true; } _inSsi = false; @@ -195,11 +177,6 @@ string const& JobQuery::getPayload() const { return _jobDescription->payload(); void JobQuery::callMarkCompleteFunc(bool success) { _markCompleteFunc->operator()(success); } -/* &&& -std::ostream& operator<<(std::ostream& os, JobQuery const& jq) { - return os << "{" << jq.getIdStr() << jq._jobDescription << " " << *jq._jobStatus << "}"; -} -*/ ostream& JobQuery::dumpOS(ostream& os) const { return os << "{" << getIdStr() << _jobDescription << " " << _jobStatus << "}"; } diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index a5be0c93ff..a4705baf46 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -46,7 +46,6 @@ 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 { class JobQuery : public JobBase { public: typedef std::shared_ptr Ptr; @@ -55,7 +54,6 @@ class JobQuery : public JobBase { static JobQuery::Ptr create(Executive::Ptr const& executive, JobDescription::Ptr const& jobDescription, 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; @@ -86,7 +84,6 @@ class JobQuery : public JobBase { } void callMarkCompleteFunc(bool success) override; - //&&& std::shared_ptr getMarkCompleteFunc() { return _markCompleteFunc; } bool cancel(bool superfluous = false); bool isQueryCancelled() override; @@ -97,16 +94,12 @@ class JobQuery : public JobBase { std::ostream& dumpOS(std::ostream& os) const override; - //&&&friend std::ostream& operator<<(std::ostream& os, JobQuery const& jq); - /// 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); - //&&&bool isCancelled() { return _cancelled; } - /// Set to true if this job is part of an UberJob void setInUberJob(bool inUberJob) { _inUberJob = inUberJob; }; @@ -114,7 +107,6 @@ class JobQuery : public JobBase { bool inUberJob() const { return _inUberJob; } protected: - //&&& void _setup() { _jobDescription->respHandler()->setJobQuery(shared_from_this()); } void _setup() { JobBase::Ptr jbPtr = shared_from_this(); _jobDescription->respHandler()->setJobQuery(jbPtr); diff --git a/src/qdisp/QueryRequest.cc b/src/qdisp/QueryRequest.cc index b0a8626314..88aa14170e 100644 --- a/src/qdisp/QueryRequest.cc +++ b/src/qdisp/QueryRequest.cc @@ -61,18 +61,6 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.qdisp.QueryRequest"); namespace lsst::qserv::qdisp { -/* &&& -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(JobBase::Ptr const& job) : _job(job), _qid(job->getQueryId()), @@ -99,18 +87,15 @@ QueryRequest::~QueryRequest() { char* QueryRequest::GetRequest(int& requestLength) { QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); lock_guard lock(_finishStatusMutex); - //&&&auto jq = _jobQuery; auto jq = _job; 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(); requestLength = jq->getPayload().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()); return const_cast(jq->getPayload().data()); } @@ -128,7 +113,6 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co } // Make a copy of the _jobQuery shared_ptr in case _jobQuery gets reset by a call to cancel() - //&&&auto jq = _jobQuery; auto jq = _job; { lock_guard lock(_finishStatusMutex); @@ -141,7 +125,6 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co ostringstream os; os << _jobIdStr << __func__ << " request failed " << getSsiErr(eInfo, nullptr) << " " << GetEndPoint(); - //&&&jq->getDescription()->respHandler()->errorFlush(os.str(), -1); jq->getRespHandler()->errorFlush(os.str(), -1); jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_ERROR, "SSI"); _errorFinish(); @@ -183,7 +166,6 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co /// 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) { bool QueryRequest::_importResultFile(JobBase::Ptr const& job) { // It's possible jq and _jobQuery differ, so need to use jq. if (job->isQueryCancelled()) { @@ -245,7 +227,6 @@ bool QueryRequest::_importResultFile(JobBase::Ptr const& job) { /// Process an incoming error. bool QueryRequest::_importError(string const& msg, int code) { - //&&&auto jq = _jobQuery; auto jq = _job; { lock_guard lock(_finishStatusMutex); @@ -266,9 +247,7 @@ void QueryRequest::ProcessResponseData(XrdSsiErrInfo const& eInfo, char* buff, i throw util::Bug(ERR_LOC, err); } -//&&&void QueryRequest::_flushError(JobQuery::Ptr const& jq) { void QueryRequest::_flushError(JobBase::Ptr const& jq) { - //&&&ResponseHandler::Error err = jq->getDescription()->respHandler()->getError(); ResponseHandler::Error err = jq->getRespHandler()->getError(); jq->getStatus()->updateInfo(_jobIdStr, JobStatus::MERGE_ERROR, "MERGE", err.getCode(), err.getMsg(), MSG_ERROR); @@ -333,57 +312,6 @@ void QueryRequest::cleanup() { shared_ptr keep(move(_keepAlive)); } -/* &&& -/// 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 job = _job; - - { - // 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(); - _finishedCalled = true; - if (!ok) { - LOGS(_log, LOG_LVL_ERROR, "QueryRequest::_errorFinish NOT ok"); - } else { - LOGS(_log, LOG_LVL_DEBUG, "QueryRequest::_errorFinish 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 until cleanup() is done. - 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 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. @@ -395,7 +323,7 @@ bool QueryRequest::_errorFinish(bool shouldCancel) { auto jbase = _job; JobQuery::Ptr jq = dynamic_pointer_cast(jbase); if (jq == nullptr) { - //&&& TODO:UJ THIS NEEDS WORK - UberJob failures are different than JobQuery failures. + //&&&uj IMPORTANT UberJob failures are different than JobQuery failures. UberJob::Ptr uberJob = dynamic_pointer_cast(jbase); if (uberJob != nullptr) { throw util::Bug(ERR_LOC, "&&&NEED_CODE for _errorFinish to work correctly with UberJob"); diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 27f1fad591..1d37064dff 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -57,7 +57,6 @@ 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() {} @@ -86,11 +85,9 @@ class ResponseHandler { /// Scrub the results from jobId-attempt from the result table. virtual void prepScrubResults(int jobId, int attempt) = 0; - //&&& std::weak_ptr getJobQuery() { return _jobQuery; } std::weak_ptr getJobBase() { return _jobBase; } private: - //&&& std::weak_ptr _jobQuery; std::weak_ptr _jobBase; }; diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index cef081363e..f5144a1de6 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -28,18 +28,24 @@ // 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 "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"); @@ -47,25 +53,6 @@ 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, string const& workerResource) { - UberJob::Ptr uJob(new UberJob(executive, respHandler, queryId, uberJobId, czarId, workerResource)); - uJob->_setup(); - return uJob; -} - -UberJob::UberJob(Executive::Ptr const& executive, - std::shared_ptr const& respHandler, - int queryId, int uberJobId, qmeta::CzarId czarId, string const& workerResource) - : JobBase(), _workerResource(workerResource), _executive(executive), - _respHandler(respHandler), _queryId(queryId), _uberJobId(uberJobId), - _czarId(czarId), _idStr("QID=" + to_string(_queryId) + ":uber=" + to_string(uberJobId)) { - _qdispPool = executive->getQdispPool(); - _jobStatus = make_shared(); -} -*/ UberJob::Ptr UberJob::create(Executive::Ptr const& executive, std::shared_ptr const& respHandler, int queryId, int uberJobId, qmeta::CzarId czarId, @@ -102,9 +89,81 @@ bool UberJob::addJob(JobQuery* job) { } bool UberJob::runUberJob() { + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() start"); + // &&&uj most, if not all, of this should be done in a command in the QDispPool. + // &&&uk especially the communication parts. QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); - // Build the uberjob payload. - // TODO:UJ For simplicity in the first pass, just make a TaskMsg for each Job and append it to the + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() a"); + // Build the uberjob payload for each job. + nlohmann::json uj; + for (auto const& jqPtr : _jobs) { + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() a1"); + jqPtr->getDescription()->incrAttemptCountScrubResultsJson(); + } + + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() b"); + // Send the uberjob to the worker + auto const method = http::Method::POST; + string const url = "http://" + _wContactInfo->wHost + ":" + to_string(_wContactInfo->wPort) + "/queryjob"; + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() c " << url); + vector const headers = {"Content-Type: application/json"}; + auto const& czarConfig = cconfig::CzarConfig::instance(); + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() c"); + // 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()}}}}; + + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() d " << request); + auto& jsUberJob = request["uberjob"]; + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() e " << jsUberJob); + auto& jsJobs = jsUberJob["jobs"]; + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() f " << jsJobs); + for (auto const& jbPtr : _jobs) { + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() f1"); + json jsJob = {{"jobdesc", *(jbPtr->getDescription()->getJsForWorker())}}; + jsJobs.push_back(jsJob); + jbPtr->getDescription()->resetJsForWorker(); // no longer needed. + } + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() g"); + LOGS(_log, LOG_LVL_WARN, __func__ << " &&&REQ " << request); + 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); + bool transmitSuccess = false; + try { + json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() response=" << response); + if (0 != response.at("success").get()) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() success"); + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, "&&&uj NEED CODE UberJob::runUberJob() success=0"); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); + } + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, "&&&uj NEED CODE UberJob::runUberJob() need to try to send jobs elsewhere"); + } else { + LOGS(_log, LOG_LVL_WARN, + "&&&uj NEED CODE UberJob::runUberJob() need to register all jobs as transmitted to worker"); + } + +#if 0 // &&&uj Everything in this block needs to happen in some manner. Where is the question + // For simplicity in the first pass, just make a TaskMsg for each Job and append it to the // UberJobMsg. // This is terribly inefficient and should be replaced by using a template and list of chunks that // the worker fills in, much like subchunks are done now. @@ -160,13 +219,16 @@ bool UberJob::runUberJob() { } LOGS(_log, LOG_LVL_WARN, "runUberJob failed. cancelled=" << cancelled << " reset=" << handlerReset << " started=" << started); +#endif // &&& return false; } void UberJob::prepScrubResults() { - throw util::Bug( - ERR_LOC, - "&&& If needed, prepScrubResults should call prepScrubResults for all JobQueries in the UberJob"); + // &&&uj There's a good chance this will not be needed as incomplete files 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, + "&&&uj If needed, should call prepScrubResults for all JobQueries in the UberJob "); } bool UberJob::isQueryCancelled() { @@ -201,7 +263,7 @@ void UberJob::callMarkCompleteFunc(bool success) { } std::ostream& UberJob::dumpOS(std::ostream& os) const { - os << "(workerResource=" << _workerResource << " jobs sz=" << _jobs.size() << "("; + os << "(jobs sz=" << _jobs.size() << "("; for (auto const& job : _jobs) { JobDescription::Ptr desc = job->getDescription(); ResourceUnit ru = desc->resource(); diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index acc3fbc12f..86262e5f29 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -25,12 +25,10 @@ // Qserv headers #include "qmeta/types.h" -#include "czar/CzarChunkMap.h" // Need nested class. &&& Make non-nested? -#include "czar/CzarRegistry.h" // Need nested class. &&& Make non-nested? +#include "czar/CzarChunkMap.h" // Need nested class. &&&uj Make non-nested? +#include "czar/CzarRegistry.h" // Need nested class. &&&uj Make non-nested? #include "qdisp/Executive.h" #include "qdisp/JobBase.h" -//&&&#include "qdisp/JobQuery.h" -//&&&#include "qdisp/QueryRequest.h" // This header declarations namespace lsst { namespace qserv { namespace qdisp { @@ -43,12 +41,6 @@ class UberJob : public JobBase { static uint32_t getMagicNumber() { return 93452; } - /* &&& - static Ptr create(Executive::Ptr const& executive, - std::shared_ptr const& respHandler, - int queryId, int uberJobId, qmeta::CzarId czarId, std::string const& workerResource); - */ - static Ptr create(Executive::Ptr const& executive, std::shared_ptr const& respHandler, int queryId, int uberJobId, qmeta::CzarId czarId, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); @@ -82,10 +74,9 @@ class UberJob : public JobBase { bool verifyPayload() const; - std::string getWorkerResource() { return _workerResource; } int getJobCount() const { return _jobs.size(); } - /// &&& TODO:UJ may not need, + /// &&&uj uj may not need, void prepScrubResults(); //&&&uj @@ -99,11 +90,6 @@ class UberJob : public JobBase { std::ostream& dumpOS(std::ostream& os) const override; private: - /* &&& - UberJob(Executive::Ptr const& executive, - std::shared_ptr const& respHandler, - int queryId, int uberJobId, qmeta::CzarId czarId, std::string const& workerResource); - */ UberJob(Executive::Ptr const& executive, std::shared_ptr const& respHandler, int queryId, int uberJobId, qmeta::CzarId czarId, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); @@ -112,7 +98,7 @@ class UberJob : public JobBase { _respHandler->setJobQuery(jbPtr); } - std::vector _jobs; + std::vector _jobs; // &&&uj should be a shared ptr ??? std::atomic _started{false}; bool _inSsi = false; JobStatus::Ptr _jobStatus; @@ -120,8 +106,7 @@ class UberJob : public JobBase { std::shared_ptr _queryRequestPtr; std::mutex _qrMtx; - std::string const _workerResource; - std::string _payload; ///< XrdSsi message to be sent to the _workerResource. + std::string _payload; ///< XrdSsi message to be sent to the _workerResource. //&&&uj remove when possible std::weak_ptr _executive; std::shared_ptr _respHandler; @@ -130,7 +115,7 @@ class UberJob : public JobBase { qmeta::CzarId _czarId; std::string const _idStr; - std::shared_ptr _qdispPool; + std::shared_ptr _qdispPool; //&&&uj needed? // &&&uj czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // &&& check if this is needed diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 45b44fc6c1..006e7efbc4 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -70,7 +70,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 diff --git a/src/qmeta/CMakeLists.txt b/src/qmeta/CMakeLists.txt index 2fbb078f56..a3c2ce25e1 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -14,23 +14,8 @@ target_link_libraries(qserv_meta PUBLIC qserv_common log mysqlclient_r - -# ccontrol -# czar -# parser -# rproc -# qana -# qproc -# query + http ) -# &&& see if there's a way to remove some target_link_libraries -# &&& ccontrol -# &&& czar -# &&& parser -# &&& rproc -# &&& qana -# &&& qproc -# &&& query install(TARGETS qserv_meta) install(DIRECTORY schema/ DESTINATION ${CMAKE_INSTALL_PREFIX}/qserv/smig/qmeta/schema/) diff --git a/src/qmeta/QMeta.h b/src/qmeta/QMeta.h index 20af80bdf6..0a215965ca 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.cc b/src/qproc/TaskMsgFactory.cc index e14931a38c..6302ead41b 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,9 +52,11 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.qproc.TaskMsgFactory"); } +using namespace std; + namespace lsst::qserv::qproc { -//&&&uj +//&&&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) { @@ -62,9 +65,7 @@ bool TaskMsgFactory::fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& resultTable = chunkResultName; } // shared - //&&&taskMsg->set_session(_session); taskMsg->set_db(chunkQuerySpec.db); - //&&&taskMsg->set_protocol(2); taskMsg->set_queryid(queryId); taskMsg->set_jobid(jobId); taskMsg->set_attemptcount(attemptCount); @@ -119,7 +120,6 @@ bool TaskMsgFactory::fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& std::shared_ptr TaskMsgFactory::_makeMsg(ChunkQuerySpec const& chunkQuerySpec, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId) { - LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::_makeMsg start"); std::string resultTable("Asdfasfd"); if (!chunkResultName.empty()) { resultTable = chunkResultName; @@ -212,4 +212,121 @@ 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) { + LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::makeMsgJson start"); + std::string resultTable("Asdfasfd"); + if (!chunkResultName.empty()) { + resultTable = + chunkResultName; // &&&uj this probably needs to be replaced with whatever it should be now. + } + + 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()}})); // &&&uj verify that these can be put in the + // uberjob to reduce copies. + + 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++) { // &&& delete block + LOGS(_log, LOG_LVL_TRACE, (sPtr->queries).at(t)); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC q=" << (sPtr->queries).at(t)); + } + for (auto const& sbi : sPtr->subChunkIds) { // &&& delete block + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC 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); + } + + LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::makeMsg end " << jsJobMsg); + 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()}}; + LOGS(_log, LOG_LVL_WARN, "&&&SUBCa jsFrag=" << jsFrag); + + string tmp("&&&SUBCa queries="); + for (auto const& qstr : queries) { + tmp += "~"; + tmp += qstr + "~"; + } + LOGS(_log, LOG_LVL_WARN, __func__ << tmp); + + tmp = "&&&SUBCa scids="; + for (auto const& qstr : subchunkIds) { + tmp += "~"; + tmp += qstr + "~"; + } + LOGS(_log, LOG_LVL_WARN, __func__ << tmp); + + auto& jsQueries = jsFrag["queries"]; + for (auto& qry : queries) { + nlohmann::json jsQry = {{"subQuery", qry}}; + jsQueries.push_back(move(jsQry)); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsQueries=" << jsQueries); + + // 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); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsSubchunkTables=" << jsSubchunkTables); + + // Add subchunk id numbers + auto& jsSubchunkIds = jsFrag["subchunkIds"]; + for (auto& subchunkId : subchunkIds) { + jsSubchunkIds.push_back(subchunkId); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsSubchunkIds=" << jsSubchunkIds); + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBCz jsFrag=" << jsFrag); + jsFragments.push_back(move(jsFrag)); +} + } // namespace lsst::qserv::qproc diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h index 493e29c605..d35ba9bdd5 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" @@ -64,14 +67,26 @@ class TaskMsgFactory { bool fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId); + // &&& doc + virtual std::shared_ptr makeMsgJson(ChunkQuerySpec const& s, + std::string const& chunkResultName, QueryId queryId, + int jobId, int attemptCount, qmeta::CzarId czarId); + private: + // &&&uj probably delete std::shared_ptr _makeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId); + // &&&uj probably delete void _addFragment(proto::TaskMsg& taskMsg, std::string const& resultName, DbTableSet const& subChunkTables, std::vector const& subChunkIds, std::vector const& queries); + + /// &&& doc + 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/xrdsvc/CMakeLists.txt b/src/xrdsvc/CMakeLists.txt index 2babecab97..072fdd99f0 100644 --- a/src/xrdsvc/CMakeLists.txt +++ b/src/xrdsvc/CMakeLists.txt @@ -6,6 +6,7 @@ target_sources(qserv_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..ca3a06244a 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 @@ -35,6 +36,7 @@ #include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" #include "mysql/MySqlUtils.h" +//&&& #include "qmeta/types.h" #include "util/String.h" #include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" @@ -48,6 +50,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 +86,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..41fd85335b 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( //&&&uj + {{"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..853251beaf --- /dev/null +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -0,0 +1,270 @@ +/* + * 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" +#include "XrdSsi/XrdSsiCluster.hh" + +// Qserv headers +#include "http/Client.h" // &&&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" // &&&uj +#include "util/String.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" + +//&&&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 +// of the corresponding Controller-side operation. +// &&& 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 + "']"; + debug(func + "&&&uj xrdsvc"); + enforceInstanceId(func, wconfig::WorkerConfig::instance()->replicationInstanceId()); + enforceWorkerId(func); + if (subModuleName == "QUERYJOB") return _queryJob(); + throw invalid_argument(context() + func + " unsupported sub-module"); +} + +json HttpWorkerCzarModule::_queryJob() { // &&&uj + //&&&debug(__func__); + debug(string(__func__) + " &&&uj _queryJob()"); + 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) { + // See qdisp::UberJob::runUberJob() for json message construction. + LOGS(_log, LOG_LVL_ERROR, __func__ << "&&&SUBC NEEDS CODE"); + auto const& jsReq = body().objJson; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsReq=" << jsReq); + string const targetWorkerId = body().required("worker"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC targetWorkerId=" << targetWorkerId); + + // &&& ??? Maybe add RequestBody(json const& js) constructor to leverage functions for nested items like + // "czar". + //&&&auto const& jsCzar = jsReq["czar"]; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC a"); + http::RequestBody rbCzar(body().required("czar")); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC b"); + auto czarName = rbCzar.required("name"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC c"); + auto czarId = rbCzar.required("id"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC d"); + auto czarPort = rbCzar.required("management-port"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC e"); + auto czarHostName = rbCzar.required("management-host-name"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC f"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC czar n=" << czarName << " id=" << czarId << " p=" << czarPort + << " h=" << czarHostName); + + http::RequestBody rbUberJob(body().required("uberjob")); + auto ujQueryId = rbUberJob.required("queryid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC g"); + auto ujId = rbUberJob.required("uberjobid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC h"); + auto ujCzarId = rbUberJob.required("czarid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC i"); + auto ujJobs = rbUberJob.required("jobs"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC j"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); + for (auto const& job : ujJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBody rbJobDesc(jsJobDesc); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jobdesc " << jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); + auto const jdCzarId = rbJobDesc.required("czarId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); + auto const jdQueryId = rbJobDesc.required("queryId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); + auto const jdJobId = rbJobDesc.required("jobId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k5"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); + auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); + auto const jdMaxTableSize = rbJobDesc.required("maxTableSize"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSize << " chunkId=" << jdChunkId); + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); + auto const jdQueryFragments = rbJobDesc.required("queryFragments"); + for (auto const& frag : jdQueryFragments) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC frag=" << frag); + http::RequestBody rbFrag(frag); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); + auto const& jsQueries = rbFrag.required("queries"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); + for (auto const& subQ : + jsQueries) { // &&&uj move to uberjob, these should be the same for all jobs + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c1"); + http::RequestBody rbSubQ(subQ); + auto const subQuery = rbSubQ.required("subQuery"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC subQuery=" << subQuery); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d1"); + auto const& resultTable = rbFrag.required("resultTable"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); + auto const& jsSubIds = rbFrag.required("subchunkIds"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId jsSubIds=" << jsSubIds); + for (auto const& scId : jsSubIds) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId=" << scId); + } + auto const& jsSubTables = rbFrag.required("subchunkTables"); + for (string scTable : jsSubTables) { // &&&uj are these the same for all jobs? + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10f1"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scTable=" << scTable); + } + } + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11"); + auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); + for (auto const& tbl : jdChunkScanTables) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a1"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC tbl=" << tbl); + http::RequestBody rbTbl(tbl); + auto const& chunkScanDb = rbTbl.required("db"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a2"); + auto const& lockInMemory = rbTbl.required("lockInMemory"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a3"); + auto const& chunkScanTable = rbTbl.required("table"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a4"); + auto const& tblScanRating = rbTbl.required("tblScanRating"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory + << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k12"); + } + + // &&&uj temporary, send response back to czar saying file is ready. The file is not ready, but this + // an initial comms test + _temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); + + json jsRet = {{"success", 1}, {"errortype", "none"}, {"note", "none"}}; + return jsRet; +} + +void HttpWorkerCzarModule::_temporaryRespFunc(string const& targetWorkerId, string const& czarName, + qmeta::CzarId czarId, string const& czarHostName, int czarPort, + uint64_t queryId, uint64_t uberJobId) { + json request = {{"version", http::MetaModule::version}, + {"workerid", foreman()->chunkInventory()->id()}, + {"auth_key", authKey()}, + {"czar", czarName}, + {"czarid", czarId}, + {"queryid", queryId}, + {"uberjobid", uberJobId}}; + + auto const method = http::Method::POST; + vector const headers = {"Content-Type: application/json"}; + string const url = "http://" + czarHostName + ":" + to_string(czarPort) + "/queryjob-ready"; + string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + http::Client client(method, url, request.dump(), headers); + bool transmitSuccess = false; + try { + json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj response=" << response); + if (0 != response.at("success").get()) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj success"); + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE success=0"); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); + } + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, + __func__ << "&&&uj NEED CODE try again??? Let czar find out through polling worker status???"); + } else { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); + } +} + +} // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/xrdsvc/HttpWorkerCzarModule.h new file mode 100644 index 0000000000..2f0ef10723 --- /dev/null +++ b/src/xrdsvc/HttpWorkerCzarModule.h @@ -0,0 +1,93 @@ +/* + * 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 { + +/// &&& doc +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); + + /// &&& doc + nlohmann::json _queryJob(); + + /// &&& doc + nlohmann::json _handleQueryJob(std::string const& func); + + /// &&&uj temporary function for testing communication. Something like this will + /// need to be called when the uberjob has finished making the result file. + void _temporaryRespFunc(std::string const& targetWorkerId, std::string const& czarName, + qmeta::CzarId czarId, std::string const& czarHostName, int czarPort, + uint64_t ujQueryId, uint64_t ujId); +}; + +} // namespace lsst::qserv::xrdsvc + +#endif // LSST_QSERV_XRDSVC_HTTPWORKERCZARMODULE_H diff --git a/src/xrdsvc/SsiRequest.cc b/src/xrdsvc/SsiRequest.cc index 19db57205c..ec295cfd1f 100644 --- a/src/xrdsvc/SsiRequest.cc +++ b/src/xrdsvc/SsiRequest.cc @@ -90,7 +90,6 @@ 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); @@ -124,7 +123,6 @@ void SsiRequest::execute(XrdSsiRequest& req) { // 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); @@ -188,7 +186,6 @@ void SsiRequest::execute(XrdSsiRequest& req) { 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 { @@ -242,7 +239,6 @@ void SsiRequest::execute(XrdSsiRequest& req) { break; } default: - LOGS(_log, LOG_LVL_WARN, "&&& SsiRequest::execute default"); reportError("Unexpected unit type '" + std::to_string(ru.unitType()) + "', resource name: " + _resourceName); break; diff --git a/src/xrdsvc/SsiService.cc b/src/xrdsvc/SsiService.cc index 90e770a7ed..473bd5f071 100644 --- a/src/xrdsvc/SsiService.cc +++ b/src/xrdsvc/SsiService.cc @@ -283,14 +283,12 @@ SsiService::~SsiService() { void SsiService::ProcessRequest(XrdSsiRequest& reqRef, XrdSsiResource& resRef) { LOGS(_log, LOG_LVL_DEBUG, "Got request call where rName is: " << resRef.rName); - LOGS(_log, LOG_LVL_WARN, "&&& SsiService::ProcessRequest start"); 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); - LOGS(_log, LOG_LVL_WARN, "&&& SsiService::ProcessRequest end"); } } // namespace lsst::qserv::xrdsvc From e6d2c2223c9a0ac2a529155a502f394bb1b32b1b Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 28 May 2024 14:14:34 -0700 Subject: [PATCH 03/15] Added new create tasks. --- src/ccontrol/UserQuerySelect.cc | 4 +- src/czar/CzarChunkMap.cc | 1 - src/czar/HttpCzarWorkerModule.cc | 2 +- src/global/intTypes.h | 1 + src/proto/ScanTableInfo.h | 6 + src/qdisp/Executive.h | 4 +- src/qdisp/UberJob.h | 4 +- src/wbase/CMakeLists.txt | 1 + src/wbase/FileChannelShared.cc | 107 +++++++++--- src/wbase/FileChannelShared.h | 21 ++- src/wbase/Task.cc | 253 ++++++++++++++++++++++++++++- src/wbase/Task.h | 48 +++++- src/wbase/UberJobData.cc | 53 ++++++ src/wbase/UberJobData.h | 75 +++++++++ src/xrdsvc/HttpWorkerCzarModule.cc | 251 +++++++++++++++------------- 15 files changed, 671 insertions(+), 160 deletions(-) create mode 100644 src/wbase/UberJobData.cc create mode 100644 src/wbase/UberJobData.h diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 8c55b8c7fd..3a5676d2c6 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -242,7 +242,7 @@ std::string UserQuerySelect::getResultQuery() const { #if 0 // &&& /// Begin running on all chunks added so far. -void UserQuerySelect::submitOld() { // &&& to be deleted +void UserQuerySelect::submit() { // &&& to be deleted _qSession->finalize(); // Using the QuerySession, generate query specs (text, db, chunkId) and then @@ -339,7 +339,6 @@ void UserQuerySelect::submit() { //&&&uj LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect beginning submission"); assert(_infileMerger); - //&&&auto taskMsgFactory = std::make_shared(_qMetaQueryId); auto taskMsgFactory = std::make_shared(); TmpTableName ttn(_qMetaQueryId, _qSession->getOriginal()); std::vector chunks; @@ -435,7 +434,6 @@ void UserQuerySelect::submit() { //&&&uj vector uberJobs; auto czarPtr = czar::Czar::getCzar(); - // auto workerResources = czarPtr->getWorkerResourceLists(); //&&& replace with CzarRegistry stuff auto czChunkMap = czarPtr->getCzarChunkMap(); auto czRegistry = czarPtr->getCzarRegistry(); diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 6f6332fec8..fac5d6eb35 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" diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 5d6ac274dc..7b945acc4f 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -60,7 +60,7 @@ json HttpCzarWorkerModule::executeImpl(string const& subModuleName) { string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; debug(func); //&&&uj this seems irrelevant for a worker enforceInstanceId(func, - //cconfig::CzarConfig::instance()->replicationInstanceId()); + // cconfig::CzarConfig::instance()->replicationInstanceId()); enforceCzarName(func); if (subModuleName == "QUERYJOB-ERROR") return _queryJobError(); diff --git a/src/global/intTypes.h b/src/global/intTypes.h index 4182544f7f..f12192becd 100644 --- a/src/global/intTypes.h +++ b/src/global/intTypes.h @@ -37,6 +37,7 @@ typedef std::vector Int32Vector; /// Typedef for Query ID in query metadata. typedef std::uint64_t QueryId; +typedef std::uint64_t UberJobId; /// Class to provide a consistent format for QueryIds in the log file class QueryIdHelper { 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/qdisp/Executive.h b/src/qdisp/Executive.h index cf34e5f51b..6060024483 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -179,7 +179,7 @@ class Executive : public std::enable_shared_from_this { void updateProxyMessages(); /// Add UbjerJobs to this user query. &&& - void addUberJobs(std::vector> const& jobsToAdd); /// &&& + void addUberJobs(std::vector> const& jobsToAdd); ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& bool startUberJob(std::shared_ptr const& uJob); /// &&& @@ -260,7 +260,7 @@ class Executive : public std::enable_shared_from_this { // Add a job to the _chunkToJobMap //&&&uj void _addToChunkJobMap(std::shared_ptr const& job); //&&&uj - /// _chunkToJobMap is created once and then destroyed when used. + /// _chunkToJobMap is created once and then destroyed after use. 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 diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 86262e5f29..6952ca0835 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -110,8 +110,8 @@ class UberJob : public JobBase { std::weak_ptr _executive; std::shared_ptr _respHandler; - int const _queryId; - int const _uberJobId; + QueryId const _queryId; + UberJobId const _uberJobId; qmeta::CzarId _czarId; std::string const _idStr; diff --git a/src/wbase/CMakeLists.txt b/src/wbase/CMakeLists.txt index b47024647f..ae1fd984a8 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 42a8814822..f411df080c 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -267,8 +267,12 @@ shared_ptr FileChannelShared::create(shared_ptr const& sendChannel, qmeta::CzarId czarId, string const& workerId) - : _sendChannel(sendChannel), + : _isUberJob(false), + _sendChannel(sendChannel), + _uberJobId(0), _czarId(czarId), + _czarHostName(""), ///< Name of the czar host. + _czarPort(-1), _workerId(workerId), _protobufArena(make_unique()), _scsId(scsSeqId++) { @@ -278,6 +282,30 @@ FileChannelShared::FileChannelShared(shared_ptr const& sendC } } +//&&&uj +FileChannelShared::Ptr FileChannelShared::create(UberJobId uberJobId, qmeta::CzarId czarId, + string const& czarHostName, int czarPort, + string const& workerId) { + lock_guard const lock(_resultsDirCleanupMtx); + return Ptr(new FileChannelShared(uberJobId, czarId, czarHostName, czarPort, workerId)); +} + +FileChannelShared::FileChannelShared(UberJobId uberJobId, qmeta::CzarId czarId, string const& czarHostName, + int czarPort, string const& workerId) + : _isUberJob(true), + _sendChannel(nullptr), + _uberJobId(uberJobId), + _czarId(czarId), + _czarHostName(czarHostName), + _czarPort(czarPort), + _workerId(workerId), + _protobufArena(make_unique()), + _scsId(scsSeqId++), + _useHttp(true) { + LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created"); + LOGS(_log, LOG_LVL_WARN, "&&& FileChannelShared created useHttp true"); +} + FileChannelShared::~FileChannelShared() { // 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 @@ -286,11 +314,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()"); + } } + } else { + LOGS(_log, LOG_LVL_WARN, "&&&uj should anything be sent to czar at this point???"); } LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared deleted"); } @@ -310,8 +342,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) { @@ -322,11 +358,16 @@ 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 { + LOGS(_log, LOG_LVL_WARN, "&&&uj NEED CODE send msg to czar with the errors"); } - return true; + return false; } bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& task, @@ -429,7 +470,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); + } + // &&&uj anything else need to be done? + return oldVal; + } } bool FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_ptr const& task, @@ -579,20 +629,27 @@ 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; - } + if (!_useHttp) { + // 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; + // 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 { + LOGS(_log, LOG_LVL_WARN, "&&&uj NEED CODE send the url back with http"); + // &&&uj the http communications need to happen in a different thread, or this thread can be booted + // from + // &&&uj the scheduler so that it can just wait for a response. } return true; } diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 0febe6f460..716000cc3c 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -115,10 +115,15 @@ class FileChannelShared { */ static nlohmann::json filesToJson(std::vector const& queryIds, unsigned int maxFiles); + //&&&uj /// The factory method for the channel class. 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(UberJobId uberJobId, qmeta::CzarId czarId, std::string const& czarHostName, + int czarPort, std::string const& workerId); + FileChannelShared() = delete; FileChannelShared(FileChannelShared const&) = delete; FileChannelShared& operator=(FileChannelShared const&) = delete; @@ -166,10 +171,14 @@ class FileChannelShared { bool isDead(); private: + //&&&uj /// Private constructor to protect shared pointer integrity. FileChannelShared(std::shared_ptr const& sendChannel, qmeta::CzarId czarId, std::string const& workerId); + FileChannelShared(UberJobId uberJobId, 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); @@ -230,9 +239,14 @@ 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. + UberJobId const _uberJobId; + qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). + std::string const _czarHostName; ///< Name of the czar host. + int const _czarPort; ///< port for the czar. + std::string const _workerId; ///< The unique identifier of the worker. // Allocatons/deletion of the data messages are managed by Google Protobuf Arena. std::unique_ptr _protobufArena; @@ -272,6 +286,9 @@ 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. + + 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 5687e2ddfb..ba4d1ee610 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" @@ -55,6 +56,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" @@ -62,6 +64,7 @@ using namespace std; using namespace std::chrono_literals; +using namespace nlohmann; namespace fs = boost::filesystem; namespace { @@ -78,6 +81,19 @@ string buildResultFilePath(shared_ptr const& taskMs return path.string(); } +std::atomic fileSeq = 0; +//&&&uj +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()) + "-" + to_string(fileSeq++) + ".proto"; + return path.string(); +} + size_t const MB_SIZE_BYTES = 1024 * 1024; } // namespace @@ -101,7 +117,7 @@ bool Task::ChunkIdGreater::operator()(Task::Ptr const& x, Task::Ptr const& y) { } string const Task::defaultUser = "qsmaster"; -IdSet Task::allIds{}; +//&&& IdSet Task::allIds{}; TaskScheduler::TaskScheduler() { auto hour = chrono::milliseconds(1h); @@ -132,7 +148,7 @@ 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()), + //&&&_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 @@ -158,10 +174,12 @@ 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(); @@ -216,9 +234,109 @@ Task::Task(TaskMsgPtr const& t, int fragmentNumber, shared_ptr co } } +/// 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), + //&&&_hasDb(t->has_db()), + //&&&_db(t->has_db() ? t->db() : ""), + _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) { + // &&&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; //&&&uj + + // 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. + LOGS(_log, LOG_LVL_WARN, "&&& Task::Task not _fragmentHasSubchunks"); + //&&&for (auto const& scanTbl : t->scantable()) { + 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 { + //&&&proto::TaskMsg_Subchunk const& sc = fragment.subchunks(); + //&&&for (int j = 0; j < sc.dbtbl_size(); j++) { + LOGS(_log, LOG_LVL_WARN, "&&& Task::Task true _fragmentHasSubchunks"); + 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(sc.dbtbl(j).db(), sc.dbtbl(j).tbl()); + dbTbls_.emplace(fDbTbl.db, fDbTbl.tbl); + LOGS(_log, LOG_LVL_TRACE, + "Task::Task subchunk fDbTbl.db=" << fDbTbl.db << " fDbTbl.tbl=" << fDbTbl.tbl); + } + //&&& IntVector sVect(sc.id().begin(), sc.id().end()); + //&&&subchunksVect_ = sVect; + // &&&uj Is the entire subchunk vector needed here, or would one subchunk this query uses be enough. + // &&&uj Need to test subchunksVect_.push_back(subchunkId);. + subchunksVect_ = fragSubchunkIds; + + /* &&&uj _db set above, sc.database() is unused worker.proto + 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_)); + LOGS(_log, LOG_LVL_WARN, + "&&& fragment b db=" << _db << ":" << _chunkId << " dbTableSet" << util::printable(dbTbls_) + << " subChunks=" << util::printable(subchunksVect_)); + } + + _dbTblsAndSubchunks = make_unique(dbTbls_, subchunksVect_); + LOGS(_log, LOG_LVL_WARN, "&&&uj Task::Task UberJob version end."); +} + Task::~Task() { + /* &&& allIds.remove(to_string(_qId) + "_" + to_string(_jId)); LOGS(_log, LOG_LVL_TRACE, "~Task() : " << allIds); + */ _userQueryInfo.reset(); UserQueryInfo::uqMapErase(_qId); @@ -272,6 +390,137 @@ vector Task::createTasks(shared_ptr const& taskMsg, } sendChannel->setTaskCount(vect.size()); + LOGS(_log, LOG_LVL_WARN, "&&& Task::createTasksForChunk end vect.sz=" << 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::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(); + LOGS(_log, LOG_LVL_WARN, "&&&uj Task::createTasksForChunk start"); + + UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); + + vector vect; + for (auto const& job : jsJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBody rbJobDesc(jsJobDesc); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jobdesc " << jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); + auto const jdCzarId = rbJobDesc.required("czarId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); + 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)); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); + auto const jdJobId = rbJobDesc.required("jobId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k5"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); + auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); + auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); + auto const jdQueryFragments = rbJobDesc.required("queryFragments"); + int fragmentNumber = 0; //&&&uj should this be 1??? Is this at all useful? + for (auto const& frag : jdQueryFragments) { + vector fragSubQueries; + vector fragSubchunkIds; + vector fragSubTables; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC frag=" << frag); + http::RequestBody rbFrag(frag); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); + auto const& jsQueries = rbFrag.required("queries"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); + // &&&uj move to uberjob???, these should be the same for all jobs + for (auto const& subQ : jsQueries) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c1"); + http::RequestBody rbSubQ(subQ); + auto const subQuery = rbSubQ.required("subQuery"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC subQuery=" << subQuery); + fragSubQueries.push_back(subQuery); + } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d1"); + auto const& resultTable = rbFrag.required("resultTable"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); + auto const& jsSubIds = rbFrag.required("subchunkIds"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId jsSubIds=" << jsSubIds); + for (auto const& scId : jsSubIds) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId=" << scId); + fragSubchunkIds.push_back(scId); + } + auto const& jsSubTables = rbFrag.required("subchunkTables"); + + for (auto const& scDbTable : jsSubTables) { // &&&uj are these the same for all jobs? + http::RequestBody rbScDbTable(scDbTable); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10f1"); + string scDb = rbScDbTable.required("scDb"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scDb=" << scDb); + string scTable = rbScDbTable.required("scTable"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scTable=" << scDbTable); + TaskDbTbl scDbTbl(scDb, scTable); + fragSubTables.push_back(scDbTbl); + } + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC fragSubQueries.sz=" << fragSubQueries.size()); + 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)); // &&& change to make_shared + 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)); // &&& change to make_shared + 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)); + } + // sendChannel->setTaskCount(vect.size()); &&& done at uberjob level now + LOGS(_log, LOG_LVL_WARN, "&&&uj Task::createTasksForChunk end vect.sz=" << vect.size()); return vect; } diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 71589c3c48..50a43de0c9 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -71,8 +71,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: @@ -82,6 +88,15 @@ class TaskQueryRunner { virtual void cancel() = 0; ///< Repeated calls to cancel() must be harmless. }; +//&&&uj +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. @@ -104,7 +119,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 { // &&&uj no longer needed void add(std::string const& id) { std::lock_guard lock(mx); _ids.insert(id); @@ -159,6 +174,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); + // &&&uj too many parameters. + // &&&uj fragmentNumber seems pointless + // &&&uj hasSubchunks seems redundant. + // &&&uj Hopefully, many are the same for all tasks and can be moved to ujData and userQueryInfo. + // &&&uj Candidates: scanInfo, maxTableSizeMb, FileChannelShared, resultsHttpPort. + // &&&uj 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(); @@ -172,6 +200,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; } @@ -206,7 +244,7 @@ class Task : public util::CommandForThreadPool { TaskState state() const { return _state; } std::string getQueryString() const; - int getQueryFragmentNum() { return _queryFragmentNum; } + //&&&int getQueryFragmentNum() { return _queryFragmentNum; } std::string const& resultFilePath() const { return _resultFilePath; } std::string const& resultFileXrootUrl() const { return _resultFileXrootUrl; } std::string const& resultFileHttpUrl() const { return _resultFileHttpUrl; } @@ -326,9 +364,9 @@ 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. + //&&& 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. /// Set of tables and vector of subchunk ids used by ChunkResourceRequest. Do not change/reset. std::unique_ptr _dbTblsAndSubchunks; diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc new file mode 100644 index 0000000000..2d58eb19c1 --- /dev/null +++ b/src/wbase/UberJobData.cc @@ -0,0 +1,53 @@ +/* + * 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 + +// Qserv headers + +// LSST headers +#include "lsst/log/Log.h" + +using namespace std; + +namespace { + +LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.UberJobData"); + +} // namespace + +namespace lsst::qserv::wbase { + +UberJobData::UberJobData(UberJobId uberJobId, qmeta::CzarId czarId, std::string czarHost, int czarPort, + uint64_t queryId, std::string const& workerId) + : _uberJobId(uberJobId), + _czarId(czarId), + _czarHost(czarHost), + _czarPort(czarPort), + _queryId(queryId), + _workerId(workerId) {} + +} // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h new file mode 100644 index 0000000000..a17ceba5c7 --- /dev/null +++ b/src/wbase/UberJobData.h @@ -0,0 +1,75 @@ +/* + * 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 + +// Qserv headers +#include "global/intTypes.h" +#include "qmeta/types.h" +#include "wbase/SendChannel.h" + +namespace lsst::qserv::wbase { + +// &&&uj doc +class UberJobData { +public: + using Ptr = std::shared_ptr; + + UberJobData() = delete; + UberJobData(UberJobData const&) = delete; + + static Ptr create(UberJobId uberJobId, qmeta::CzarId czarId, std::string const& czarHost, int czarPort, + uint64_t queryId, std::string const& workerId) { + return Ptr(new UberJobData(uberJobId, czarId, czarHost, czarPort, queryId, workerId)); + } + + 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; } + +private: + UberJobData(UberJobId uberJobId, qmeta::CzarId czarId, std::string czarHost, int czarPort, + uint64_t queryId, std::string const& workerId); + + UberJobId const _uberJobId; + qmeta::CzarId const _czarId; + std::string const _czarHost; + int const _czarPort; + uint64_t const _queryId; + std::string const& _workerId; //&&&uj should be able to get this from the worker in a reasonable way. +}; + +} // namespace lsst::qserv::wbase + +#endif // LSST_QSERV_WBASE_UBERJOBDATA_H diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 853251beaf..88ab12ae00 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -29,7 +29,6 @@ // Third party headers #include "lsst/log/Log.h" -#include "XrdSsi/XrdSsiCluster.hh" // Qserv headers #include "http/Client.h" // &&&uj will probably need to be removed @@ -40,6 +39,9 @@ #include "mysql/MySqlUtils.h" #include "qmeta/types.h" // &&&uj #include "util/String.h" +#include "wbase/FileChannelShared.h" +#include "wbase/Task.h" +#include "wbase/UberJobData.h" #include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" #include "wcontrol/ResourceMonitor.h" @@ -101,132 +103,147 @@ json HttpWorkerCzarModule::_queryJob() { // &&&uj } json HttpWorkerCzarModule::_handleQueryJob(string const& func) { - // See qdisp::UberJob::runUberJob() for json message construction. - LOGS(_log, LOG_LVL_ERROR, __func__ << "&&&SUBC NEEDS CODE"); - auto const& jsReq = body().objJson; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsReq=" << jsReq); - string const targetWorkerId = body().required("worker"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC targetWorkerId=" << targetWorkerId); - - // &&& ??? Maybe add RequestBody(json const& js) constructor to leverage functions for nested items like - // "czar". - //&&&auto const& jsCzar = jsReq["czar"]; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC a"); - http::RequestBody rbCzar(body().required("czar")); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC b"); - auto czarName = rbCzar.required("name"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC c"); - auto czarId = rbCzar.required("id"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC d"); - auto czarPort = rbCzar.required("management-port"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC e"); - auto czarHostName = rbCzar.required("management-host-name"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC f"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC czar n=" << czarName << " id=" << czarId << " p=" << czarPort - << " h=" << czarHostName); + json jsRet; + vector ujTasks; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + LOGS(_log, LOG_LVL_ERROR, __func__ << "&&&SUBC NEEDS CODE"); + auto const& jsReq = body().objJson; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsReq=" << jsReq); + string const targetWorkerId = body().required("worker"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC targetWorkerId=" << targetWorkerId); - http::RequestBody rbUberJob(body().required("uberjob")); - auto ujQueryId = rbUberJob.required("queryid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC g"); - auto ujId = rbUberJob.required("uberjobid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC h"); - auto ujCzarId = rbUberJob.required("czarid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC i"); - auto ujJobs = rbUberJob.required("jobs"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC j"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); + // &&& ??? Maybe add RequestBody(json const& js) constructor to leverage functions for nested items + // like "czar". + //&&&auto const& jsCzar = jsReq["czar"]; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC a"); + http::RequestBody rbCzar(body().required("czar")); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC b"); + auto czarName = rbCzar.required("name"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC c"); + auto czarId = rbCzar.required("id"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC d"); + auto czarPort = rbCzar.required("management-port"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC e"); + auto czarHostName = rbCzar.required("management-host-name"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC f"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC czar n=" << czarName << " id=" << czarId << " p=" << czarPort + << " h=" << czarHostName); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); - for (auto const& job : ujJobs) { - json const& jsJobDesc = job["jobdesc"]; - http::RequestBody rbJobDesc(jsJobDesc); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jobdesc " << jsJobDesc); - // See qproc::TaskMsgFactory::makeMsgJson for message construction. - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); - auto const jdCzarId = rbJobDesc.required("czarId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); - auto const jdQueryId = rbJobDesc.required("queryId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); - auto const jdJobId = rbJobDesc.required("jobId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); - auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k5"); - auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); - auto const jdScanPriority = rbJobDesc.required("scanPriority"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); - auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); - auto const jdMaxTableSize = rbJobDesc.required("maxTableSize"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); - auto const jdChunkId = rbJobDesc.required("chunkId"); + http::RequestBody rbUberJob(body().required("uberjob")); + auto ujQueryId = rbUberJob.required("queryid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC g"); + auto ujId = rbUberJob.required("uberjobid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC h"); + auto ujCzarId = rbUberJob.required("czarid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC i"); + auto ujJobs = rbUberJob.required("jobs"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC j"); LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId - << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive - << " maxTblSz=" << jdMaxTableSize << " chunkId=" << jdChunkId); + __func__ << "&&&SUBC uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); - auto const jdQueryFragments = rbJobDesc.required("queryFragments"); - for (auto const& frag : jdQueryFragments) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC frag=" << frag); - http::RequestBody rbFrag(frag); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); - auto const& jsQueries = rbFrag.required("queries"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); - for (auto const& subQ : - jsQueries) { // &&&uj move to uberjob, these should be the same for all jobs - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c1"); - http::RequestBody rbSubQ(subQ); - auto const subQuery = rbSubQ.required("subQuery"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC subQuery=" << subQuery); - } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d1"); - auto const& resultTable = rbFrag.required("resultTable"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); - auto const& jsSubIds = rbFrag.required("subchunkIds"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId jsSubIds=" << jsSubIds); - for (auto const& scId : jsSubIds) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId=" << scId); - } - auto const& jsSubTables = rbFrag.required("subchunkTables"); - for (string scTable : jsSubTables) { // &&&uj are these the same for all jobs? - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10f1"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scTable=" << scTable); + //&&&uj make UberJobData, FileChannelShared, and Tasks. + auto ujData = + wbase::UberJobData::create(ujId, czarId, czarHostName, czarPort, ujQueryId, targetWorkerId); + auto channelShared = + wbase::FileChannelShared::create(ujId, czarId, czarHostName, czarPort, targetWorkerId); + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); + for (auto const& job : ujJobs) { + json const& jsJobDesc = job["jobdesc"]; + http::RequestBody rbJobDesc(jsJobDesc); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jobdesc " << jsJobDesc); + // See qproc::TaskMsgFactory::makeMsgJson for message construction. + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); + auto const jdCzarId = rbJobDesc.required("czarId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); + auto const jdQueryId = rbJobDesc.required("queryId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); + auto const jdJobId = rbJobDesc.required("jobId"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); + auto const jdAttemptCount = rbJobDesc.required("attemptCount"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k5"); + auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); + auto const jdScanPriority = rbJobDesc.required("scanPriority"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); + auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); + auto const jdMaxTableSize = rbJobDesc.required("maxTableSize"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); + auto const jdChunkId = rbJobDesc.required("chunkId"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSize << " chunkId=" << jdChunkId); + + //&&&uj need scan table info befor making tasks + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11"); + //&&&proto::ScanTableInfo::ListOf scanTables; + proto::ScanInfo scanInfo; + auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); + for (auto const& tbl : jdChunkScanTables) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a1"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC tbl=" << tbl); + http::RequestBody rbTbl(tbl); + auto const& chunkScanDb = rbTbl.required("db"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a2"); + auto const& lockInMemory = rbTbl.required("lockInMemory"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a3"); + auto const& chunkScanTable = rbTbl.required("table"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a4"); + auto const& tblScanRating = rbTbl.required("tblScanRating"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory + << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); + scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, tblScanRating); } + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k12"); + scanInfo.scanRating = jdScanPriority; + + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); + + // 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()); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11"); - auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); - for (auto const& tbl : jdChunkScanTables) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a1"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC tbl=" << tbl); - http::RequestBody rbTbl(tbl); - auto const& chunkScanDb = rbTbl.required("db"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a2"); - auto const& lockInMemory = rbTbl.required("lockInMemory"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a3"); - auto const& chunkScanTable = rbTbl.required("table"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a4"); - auto const& tblScanRating = rbTbl.required("tblScanRating"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory - << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); +#if 0 /// &&&&&&&& + // 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. //&&&uj next + t.stop(); + LOGS(_log, LOG_LVL_DEBUG, + "Enqueued TaskMsg for " << ru << " in " << t.getElapsed() << " seconds"); + break; } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k12"); - } +#endif /// &&&&&&&& + + // &&&uj temporary, send response back to czar saying file is ready. The file is not ready, but this + // is just an initial comms test + _temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); - // &&&uj temporary, send response back to czar saying file is ready. The file is not ready, but this - // an initial comms test - _temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); + string note = string("qId=") + to_string(ujQueryId) + " ujId=" + to_string(ujId) + + "tasks in uberJob=" + to_string(channelShared->getTaskCount()); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC note=" << note); + jsRet = {{"success", 1}, {"errortype", "none"}, {"note", note}}; - json jsRet = {{"success", 1}, {"errortype", "none"}, {"note", "none"}}; + } 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; } From 76be3b341365fdc647346bb84d3e78a298ae0de2 Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 5 Jun 2024 16:16:15 -0700 Subject: [PATCH 04/15] Some integration test success. --- src/ccontrol/MergingHandler.cc | 261 +++++++++++++++++++++++++++++ src/ccontrol/MergingHandler.h | 14 ++ src/ccontrol/UserQuerySelect.cc | 9 +- src/czar/HttpCzarWorkerModule.cc | 64 ++++++- src/czar/HttpCzarWorkerModule.h | 3 + src/global/intTypes.h | 5 +- src/qdisp/Executive.cc | 73 +++++++- src/qdisp/Executive.h | 40 +++-- src/qdisp/JobBase.h | 9 +- src/qdisp/JobDescription.cc | 2 +- src/qdisp/JobDescription.h | 8 +- src/qdisp/JobQuery.cc | 10 +- src/qdisp/JobQuery.h | 4 +- src/qdisp/QueryRequest.cc | 2 +- src/qdisp/QueryRequest.h | 2 +- src/qdisp/ResponseHandler.h | 7 + src/qdisp/UberJob.cc | 252 +++++++++++++++++++++++++++- src/qdisp/UberJob.h | 44 +++-- src/rproc/InfileMerger.cc | 122 +++++++++++++- src/rproc/InfileMerger.h | 5 + src/wbase/FileChannelShared.cc | 59 ++++++- src/wbase/FileChannelShared.h | 35 +++- src/wbase/UberJobData.cc | 89 +++++++++- src/wbase/UberJobData.h | 53 +++++- src/wbase/UserQueryInfo.cc | 7 + src/wbase/UserQueryInfo.h | 9 + src/wsched/ChunkTasksQueue.cc | 36 ++++ src/wsched/ScanScheduler.cc | 28 ++-- src/xrdsvc/HttpWorkerCzarModule.cc | 32 +++- 29 files changed, 1180 insertions(+), 104 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 23325b53e5..7d108ab30e 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -50,6 +50,7 @@ #include "proto/worker.pb.h" #include "qdisp/CzarStats.h" #include "qdisp/JobQuery.h" +#include "qdisp/UberJob.h" #include "rproc/InfileMerger.h" #include "util/Bug.h" #include "util/common.h" @@ -375,6 +376,171 @@ bool readHttpFileAndMerge(string const& httpUrl, return success; } +std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, + function const& messageIsReady, + shared_ptr const& httpConnPool) { + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp a"); + 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. + 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; + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp b"); + 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 + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp c"); + http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, + httpConnPool); + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp d"); + 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_WARN, "&&& readHttpFileAndMergeHttp e"); + while ((next < end) && !last) { + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e1"); + if (msgSizeBytes == 0) { + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e2"); + // 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)) { + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e3"); + // Done reading the frame header. + msgSizeBufNext = 0; + // Parse and evaluate the message length. + msgSizeBytes = *(reinterpret_cast(msgSizeBuf.data())); + if (msgSizeBytes == 0) { + throw runtime_error(context + "message size is 0 at offset " + + to_string(offset - sizeof(uint32_t)) + ", file: " + httpUrl); + } + if (msgSizeBytes > ProtoHeaderWrap::PROTOBUFFER_HARD_LIMIT) { + throw runtime_error(context + "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. + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e4"); + 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) { + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e5"); + // 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. + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e6"); + mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e7"); + if (!mergeSuccess) { + success = false; + throw runtime_error(context + "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; + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e8"); + } + } + } + }); + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e9"); + if (msgSizeBufNext != 0) { + throw runtime_error(context + "short read of the message header at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e10"); + if (msgBufNext != 0) { + throw runtime_error(context + "short read of the message body at offset " + + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); + } + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e11"); + } catch (exception const& ex) { // &&&uj anything being caught here besides runtime_error? + LOGS(_log, LOG_LVL_ERROR, ex.what()); + success = false; + } + + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp f"); + // 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); + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp g"); + remover.read([](char const* inBuf, size_t inBufSize) {}); + LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp h"); + } 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_WARN, "&&& readHttpFileAndMergeHttp end succes=" << success << " mergeSuccess=" << mergeSuccess); + return {success, mergeSuccess}; +} + + } // namespace namespace lsst::qserv::ccontrol { @@ -510,10 +676,105 @@ bool MergingHandler::_merge(proto::ResponseSummary const& responseSummary, 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, "_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; + /// &&&uj NEED CODE + //&&& _wName = responseSummary.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 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 jobQuery = std::dynamic_pointer_cast(jobBase); + auto const uberJob = std::dynamic_pointer_cast(jobBase); + + LOGS(_log, LOG_LVL_TRACE, + "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() + << " fileUrl=" << fileUrl); + + /* &&& errors will be handled by MergingHandler::flushHttpError() + if (responseSummary.errorcode() != 0 || !responseSummary.errormsg().empty()) { + _error = util::Error(responseSummary.errorcode(), responseSummary.errormsg(), + util::ErrorCode::MYSQLEXEC); + _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); + LOGS(_log, LOG_LVL_ERROR, + "MergingHandler::" << __func__ << " error from worker:" << responseSummary.wname() + << " error: " << _error); + return false; + } + */ + + /* &&&& + // 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 size, bool& last) { + last = true; + proto::ResponseData responseData; + if (responseData.ParseFromArray(buf, size) && responseData.IsInitialized()) { + bool const success = _merge(responseSummary, responseData, jobQuery); + if (success) { + resultRows += responseData.row_size(); + last = resultRows >= responseSummary.rowcount(); + } + return success; + } + throw runtime_error("MergingHandler::flush ** message deserialization failed **"); + }; + */ + // 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_WARN, "&&& dataMergerHttp "); + last = true; + proto::ResponseData responseData; + if (responseData.ParseFromArray(buf, bufSize) && responseData.IsInitialized()) { + bool const mergeSuccess = _mergeHttp(uberJob, responseData); + //&&&if (responseData.IsInitialized()) { + //&&&bool const mergeSuccess = _mergeHttp(uberJob, responseData); + if (mergeSuccess) { + resultRows += responseData.row_size(); + //&&&last = resultRows >= responseSummary.rowcount(); + last = resultRows >= expectedRows; + } + return mergeSuccess; + } + throw runtime_error("MergingHandler::flush ** message deserialization failed **"); + }; + + tie(success, shouldCancel) = ::readHttpFileAndMergeHttp(uberJob, fileUrl, dataMergerHttp, + MergingHandler::_getHttpConnPool()); + + if (success) { + _infileMerger->mergeCompleteFor(uberJob->getJobId()); + } + return {success, shouldCancel}; +} + + + } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 97cd564dd6..c8f4c3c538 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -44,6 +44,7 @@ class ResponseSummary; namespace lsst::qserv::qdisp { class JobQuery; +class UberJob; } // namespace lsst::qserv::qdisp namespace lsst::qserv::rproc { @@ -74,6 +75,16 @@ class MergingHandler : public qdisp::ResponseHandler { /// @return true if successful (no error) bool flush(proto::ResponseSummary const& responseSummary, uint32_t& resultRows) override; + /// &&&uj doc see ResponseHandler::flushHttp + /// @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. + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, uint64_t& resultRows) override; + + /// &&&uj doc + bool flushHttpError(); + /// Signal an unrecoverable error condition. No further calls are expected. void errorFlush(std::string const& msg, int code) override; @@ -101,6 +112,9 @@ class MergingHandler : public qdisp::ResponseHandler { bool _merge(proto::ResponseSummary const& responseSummary, proto::ResponseData const& responseData, std::shared_ptr const& jobQuery); + /// &&&uj doc + 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 3a5676d2c6..004179554f 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -343,7 +343,7 @@ void UserQuerySelect::submit() { //&&&uj TmpTableName ttn(_qMetaQueryId, _qSession->getOriginal()); std::vector chunks; std::mutex chunksMtx; - int sequence = 0; + JobId sequence = 0; LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew b"); auto queryTemplates = _qSession->makeQueryTemplates(); @@ -429,7 +429,8 @@ void UserQuerySelect::submit() { //&&&uj } LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e"); - if (uberJobsEnabled || true) { + //&&&if (uberJobsEnabled || true) { + if (uberJobsEnabled) { LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); vector uberJobs; @@ -677,6 +678,10 @@ void UserQuerySelect::discard() { return; } } + + // Remove _executive from ExecutiveMap &&&uj + _executive->removeFromMap(); + // Make sure resources are released. if (_executive && _executive->getNumInflight() > 0) { throw UserQueryError(getQueryIdString() + " Executive unfinished, cannot discard"); diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 7b945acc4f..7361665c61 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -28,6 +28,8 @@ // Qserv headers #include "cconfig/CzarConfig.h" +#include "qdisp/Executive.h" +#include "qdisp/UberJob.h" #include "global/intTypes.h" #include "http/Exceptions.h" #include "http/RequestQuery.h" @@ -58,6 +60,7 @@ HttpCzarWorkerModule::HttpCzarWorkerModule(string const& context, shared_ptrreplicationInstanceId()); @@ -81,9 +84,64 @@ json HttpCzarWorkerModule::_queryJobReady() { debug(__func__); checkApiVersion(__func__, 34); LOGS(_log, LOG_LVL_INFO, __func__ << "&&&uj queryJobReady json=" << body().objJson); //&&& - //&&&uj NEED CODE for this - json ret = {{"success", 1}}; - return json::object(); + auto ret = _handleJobReady(__func__); + return ret; + + //&&& json ret = {{"success", 1}}; + //&&&return json::object(); } +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", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; + try { + // See qdisp::UberJob::runUberJob() for json message construction. + auto const& js = body().objJson; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR js=" << js); + string const targetWorkerId = body().required("workerid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR targetWorkerId=" << targetWorkerId); + string const czarName = body().required("czar"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarName=" << czarName); + qmeta::CzarId const czarId = body().required("czarid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarId=" << czarId); + QueryId const queryId = body().required("queryid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR queryId=" << queryId); + UberJobId const uberJobId = body().required("uberjobid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR uberJobId=" << uberJobId); + string const fileUrl = body().required("fileUrl"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR fileUrl=" << fileUrl); + uint64_t const rowCount = body().required("rowCount"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR rowCount=" << rowCount); + uint64_t const fileSize = body().required("fileSize"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR fileSize=" << fileSize); + + + // Find UberJob + qdisp::Executive::Ptr exec = qdisp::Executive::getExecutiveFromMap(queryId); + if (exec == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobRead No executive for qid=") + to_string(queryId)); + } + qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); + if (uj == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobRead No UberJob for qid=") + to_string(queryId) + " ujId=" + to_string(uberJobId)); + } + // &&&uj NEED CODE to verify incoming values to those in the UberJob + + + + + auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); // &&&uj move this to after parse. + jsRet = importRes; + + + } catch (std::invalid_argument const& iaExp) { + LOGS(_log, LOG_LVL_ERROR, "HttpCzarWorkerModule::_handleJobReady received " << iaExp.what() << " js=" << body().objJson); + jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaExp.what()}}; + } + return jsRet; +} + + } // namespace lsst::qserv::czar diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h index 8871162b0c..8efac50c6a 100644 --- a/src/czar/HttpCzarWorkerModule.h +++ b/src/czar/HttpCzarWorkerModule.h @@ -69,6 +69,9 @@ class HttpCzarWorkerModule : public czar::HttpModule { /// &&& doc nlohmann::json _queryJobReady(); + + /// &&& doc + nlohmann::json _handleJobReady(std::string const& func); }; } // namespace lsst::qserv::czar diff --git a/src/global/intTypes.h b/src/global/intTypes.h index f12192becd..8074009d05 100644 --- a/src/global/intTypes.h +++ b/src/global/intTypes.h @@ -37,7 +37,8 @@ typedef std::vector Int32Vector; /// Typedef for Query ID in query metadata. typedef std::uint64_t QueryId; -typedef std::uint64_t UberJobId; +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 { @@ -54,7 +55,7 @@ class QueryIdHelper { /// @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/qdisp/Executive.cc b/src/qdisp/Executive.cc index 94c694373d..b433b2b9d0 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -97,6 +97,10 @@ string getErrorText(XrdSsiErrInfo& e) { namespace lsst::qserv::qdisp { +mutex Executive::_executiveMapMtx; ///< protects _executiveMap +map Executive::_executiveMap; ///< Map of executives for queries in progress. + + //////////////////////////////////////////////////////////////////////// // class Executive implementation //////////////////////////////////////////////////////////////////////// @@ -167,11 +171,50 @@ 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); + + // Insert into the global executive map. + { + lock_guard lgMap(_executiveMapMtx); + _executiveMap[_id] = shared_from_this(); + } qdisp::CzarStats::get()->trackQueryProgress(_id); } +void Executive::removeFromMap() { + if (_queryIdSet) { + return; + } + // Remove this from the global executive map. + lock_guard lgMap(_executiveMapMtx); + auto iter = _executiveMap.find(_id); + if (iter != _executiveMap.end()) { + _executiveMap.erase(iter); + } +} + +Executive::Ptr Executive::getExecutiveFromMap(QueryId qId) { + lock_guard lgMap(_executiveMapMtx); + auto iter = _executiveMap.find(qId); + if (iter == _executiveMap.end()) { + return nullptr; + } + return iter->second; +} + +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) { @@ -184,7 +227,7 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { 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()); @@ -199,7 +242,7 @@ 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; } @@ -213,7 +256,7 @@ 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()); /* &&&uj //&&&uj code just returns the jobQuery at this point, it doesn't call runJob(). LOGS(_log, LOG_LVL_DEBUG, "Executive::add with path=" << jobDesc->resource().path()); @@ -230,7 +273,7 @@ void Executive::runJobQuery(JobQuery::Ptr const& jobQuery) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::runJobQuery start"); bool started = jobQuery->runJob(); if (!started && isLimitRowComplete()) { - markCompleted(jobQuery->getIdInt(), false); + markCompleted(jobQuery->getJobId(), false); } LOGS(_log, LOG_LVL_WARN, "&&& Executive::runJobQuery end"); } @@ -246,6 +289,17 @@ void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueJobStart end"); } +void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect start"); + + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 3); + } else { + _qdispPool->queCmd(cmd, 4); + } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect end"); +} + void Executive::runUberJob(std::shared_ptr const& uberJob) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); bool started = uberJob->runUberJob(); @@ -315,9 +369,10 @@ Executive::ChunkIdJobMapType& Executive::getChunkJobMapAndInvalidate() { // &&& } void Executive::addUberJobs(std::vector> const& uJobsToAdd) { // &&& - lock_guard lck(_uberJobsMtx); + lock_guard lck(_uberJobsMapMtx); for (auto const& uJob : uJobsToAdd) { - _uberJobs.push_back(uJob); + UberJobId ujId = uJob->getJobId(); + _uberJobsMap[ujId] = uJob; } } @@ -336,7 +391,7 @@ bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& JobQuery::Ptr Executive::getSharedPtrForRawJobPtr(JobQuery* jqRaw) { //&&& assert(jqRaw != nullptr); - int jobId = jqRaw->getIdInt(); + int jobId = jqRaw->getJobId(); lock_guard lockJobMap(_jobMapMtx); auto iter = _jobMap.find(jobId); if (iter == _jobMap.end()) { @@ -350,7 +405,7 @@ JobQuery::Ptr Executive::getSharedPtrForRawJobPtr(JobQuery* jqRaw) { //&&& /// 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(); @@ -393,7 +448,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); diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 6060024483..14f85d161c 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -51,7 +51,7 @@ #include "util/ThreadPool.h" //&&& replace with better enable/disable feature. -#define uberJobsEnabled 0 //&&& +#define uberJobsEnabled 1 //&&&uj || true // Forward declarations class XrdSsiService; @@ -89,14 +89,14 @@ 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; //&&&uj This type is probably not needed - typedef std::map ChunkIdJobMapType; + //&&&typedef std::map ChunkIdJobMapType; + typedef std::map> ChunkIdJobMapType; /// Construct an Executive. /// If c->serviceUrl == ExecutiveConfig::getMockStr(), then use XrdSsiServiceMock @@ -109,10 +109,19 @@ class Executive : public std::enable_shared_from_this { ~Executive(); + /// &&& doc + static Ptr getExecutiveFromMap(QueryId qId); + + /// &&& doc + void removeFromMap(); + + /// &&& doc + std::shared_ptr findUberJob(UberJobId ujId); + /// Add an item with a reference number std::shared_ptr add(JobDescription::Ptr const& s); - /// &&& doc + /// &&& doc - to be deleted void runJobQuery(std::shared_ptr const& jobQuery); // &&&uj doc @@ -121,6 +130,9 @@ class Executive : public std::enable_shared_from_this { /// Queue a job to be sent to a worker so it can be started. void queueJobStart(PriorityCommand::Ptr const& cmd); + /// &&& doc + 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(); @@ -130,7 +142,7 @@ 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(); @@ -264,8 +276,11 @@ class Executive : public std::enable_shared_from_this { 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 + + /// Map of all UberJobs. Failed UberJobs remain in the map as new ones are created + /// to handle failed UberJobs. + std::map> _uberJobsMap; + std::mutex _uberJobsMapMtx; ///< protects _uberJobs. //&&&uj /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. @@ -281,13 +296,18 @@ 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. + + static std::mutex _executiveMapMtx; ///< protects _executiveMap + static std::map _executiveMap; ///< Map of executives for queries in progress. }; 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) { @@ -299,7 +319,7 @@ class MarkCompleteFunc { private: std::weak_ptr _executive; - int _jobId; + JobId _jobId; }; } // namespace qdisp diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h index 787975dc95..cce8d31ead 100644 --- a/src/qdisp/JobBase.h +++ b/src/qdisp/JobBase.h @@ -31,13 +31,17 @@ // This header declarations namespace lsst { namespace qserv { namespace qdisp { +class Executive; class JobStatus; class QdispPool; class ResponseHandler; class QueryRequest; /// Base class for JobQuery and UberJob. -/// TODO:UJ This could use a lot of cleanup. +/// 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; @@ -48,7 +52,7 @@ class JobBase : public std::enable_shared_from_this { virtual ~JobBase() = default; virtual QueryId getQueryId() const = 0; - virtual int getIdInt() 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 @@ -58,6 +62,7 @@ class JobBase : public std::enable_shared_from_this { 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; diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 4c6741fdb6..86efcea1ab 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -48,7 +48,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, diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 2e40de8120..2ec0d6cacf 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -62,7 +62,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, @@ -77,7 +77,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; } @@ -102,14 +102,14 @@ class JobDescription { 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 diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 3be21911c6..6f93bdf1dd 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -53,7 +53,7 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j _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); } @@ -64,7 +64,7 @@ JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_DEBUG, "~JobQuery"); } * @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()); + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_DEBUG, " runJob " << *this); auto executive = _executive.lock(); if (executive == nullptr) { @@ -120,7 +120,7 @@ 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) { lock_guard lock(_rmutex); @@ -148,7 +148,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(); @@ -164,7 +164,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"); diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index a4705baf46..32feee158f 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -65,7 +65,7 @@ class JobQuery : public JobBase { bool runJob(); QueryId getQueryId() const override { return _qid; } - int getIdInt() const override { return _jobDescription->id(); } + 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(); } @@ -88,7 +88,7 @@ class JobQuery : public JobBase { bool cancel(bool superfluous = false); bool isQueryCancelled() override; - Executive::Ptr getExecutive() { return _executive.lock(); } + std::shared_ptr getExecutive() override { return _executive.lock(); } std::shared_ptr getQdispPool() override { return _qdispPool; } diff --git a/src/qdisp/QueryRequest.cc b/src/qdisp/QueryRequest.cc index 88aa14170e..1ab2559a91 100644 --- a/src/qdisp/QueryRequest.cc +++ b/src/qdisp/QueryRequest.cc @@ -64,7 +64,7 @@ namespace lsst::qserv::qdisp { QueryRequest::QueryRequest(JobBase::Ptr const& job) : _job(job), _qid(job->getQueryId()), - _jobid(job->getIdInt()), + _jobid(job->getJobId()), _jobIdStr(job->getIdStr()), _qdispPool(_job->getQdispPool()) { QSERV_LOGCONTEXT_QUERY_JOB(_qid, _jobid); diff --git a/src/qdisp/QueryRequest.h b/src/qdisp/QueryRequest.h index 6db583ada5..b6bbd2c2c7 100644 --- a/src/qdisp/QueryRequest.h +++ b/src/qdisp/QueryRequest.h @@ -159,7 +159,7 @@ class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this _keepAlive; ///< Used to keep this object alive during race condition. QueryId _qid = 0; // for logging - int _jobid = -1; // for logging + JobId _jobid = -1; // for logging std::string _jobIdStr{QueryIdHelper::makeIdStr(0, 0, true)}; ///< for debugging only. std::atomic _finishedCalled{false}; diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 1d37064dff..afffc21e93 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -66,6 +66,13 @@ class ResponseHandler { /// @return true if successful (no error) virtual bool flush(proto::ResponseSummary const& responseSummary, uint32_t& resultRows) = 0; + /// &&&uj doc + /// @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; + /// 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 f5144a1de6..3536ae4860 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -77,11 +77,17 @@ UberJob::UberJob(Executive::Ptr const& executive, std::shared_ptr(); } -bool UberJob::addJob(JobQuery* job) { +void UberJob::_setup() { + JobBase::Ptr jbPtr = shared_from_this(); + _respHandler->setJobQuery(jbPtr); +} + +bool UberJob::addJob(JobQuery::Ptr const& job) { bool success = false; if (job->inUberJob()) { throw util::Bug(ERR_LOC, string("job already in UberJob job=") + job->dump() + " uberJob=" + dump()); } + lock_guard lck(_jobsMtx); _jobs.push_back(job); job->setInUberJob(true); success = true; @@ -92,10 +98,11 @@ bool UberJob::runUberJob() { LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() start"); // &&&uj most, if not all, of this should be done in a command in the QDispPool. // &&&uk especially the communication parts. - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getIdInt()); + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() a"); // Build the uberjob payload for each job. nlohmann::json uj; + unique_lock jobsLock(_jobsMtx); for (auto const& jqPtr : _jobs) { LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() a1"); jqPtr->getDescription()->incrAttemptCountScrubResultsJson(); @@ -136,6 +143,8 @@ bool UberJob::runUberJob() { jsJobs.push_back(jsJob); jbPtr->getDescription()->resetJsForWorker(); // no longer needed. } + jobsLock.unlock(); + LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() g"); LOGS(_log, LOG_LVL_WARN, __func__ << " &&&REQ " << request); string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; @@ -232,7 +241,7 @@ void UberJob::prepScrubResults() { } bool UberJob::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"); @@ -250,18 +259,253 @@ bool UberJob::verifyPayload() const { return true; } -void UberJob::callMarkCompleteFunc(bool success) { +void UberJob::callMarkCompleteFunc(bool success) { // &&&uj make private LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); if (!success) { + // &&&uj this function should probably only be called for successful completion. throw util::Bug(ERR_LOC, "&&&NEED_CODE may need code to properly handle failed uberjob"); } + + lock_guard lck(_jobsMtx); for (auto&& job : _jobs) { string idStr = job->getIdStr(); job->getStatus()->updateInfo(idStr, JobStatus::COMPLETE, "COMPLETE"); job->callMarkCompleteFunc(success); } + + // No longer need these here. Executive should still have copies. + _jobs.clear(); + + //&&&uj NEED CODE _resultStatus = MERGESUCCESS; + //&&&uj NEED CODE _status = COMPLETE; +} + + +/// 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(JobBase::Ptr const& job) { +nlohmann::json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile a"); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); + + //&&&uj NEED CODE update status for each job in this uberjob + // jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); + + // It's possible jq and _jobQuery differ, so need to use jq. + if (isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile import job was cancelled."); + return _errorFinish(true); + } + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile b"); + /* &&& + auto jq = std::dynamic_pointer_cast(job); + if (jq == nullptr) { + throw util::Bug(ERR_LOC, string(__func__) + " unexpected pointer type for job"); + } + */ + auto exec = _executive.lock(); + if (exec == nullptr || exec->getCancelled()) { + LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile no executive or cancelled"); + return _errorFinish(true); + } + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile c"); + + 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 _errorFinish(false); + } + + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile d"); + //&&& int messageSize = 0; + //&&& const char* message = GetMetadata(messageSize); + + LOGS(_log, LOG_LVL_DEBUG, __func__ << " _jobIdStr=" << getIdStr() << ", fileSize=" << fileSize); + + JobBase::Ptr jBaseThis = shared_from_this(); + weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); + + /// &&&&&&&&&&&&&&&&&&&&&&uj This NEEDS CODE Command class item instead of lambda and queue that to qdisppool &&&&&&&&&&&&&&&&& + /// &&&&&&&&&uj Also, HttpCzarWorkerModule::_handleJobReady isn't getting message from the worker UberJobData::fileReadyResponse &&&&&&&&& + auto fileCollectFunc = [ujThis, fileUrl, rowCount](util::CmdData*) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc a"); + /* &&& + // &&&uj this version of flush is going to have issues. + // &&&uj the reading of the file needs to happen elsewhere. + uint32_t resultRows = 0; + if (!jq->getDescription()->respHandler()->flush(responseSummary, resultRows)) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " not flushOk"); + _flushError(jq); + return false; + } + //&&&_totalRows += resultRows; + * + */ + 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_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc b"); + if (!flushSuccess) { + // This would probably indicate malformed file+rowCount or + // writing the result table failed. + ujPtr->_errorFinish(flushShouldCancel); + } + + + // 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"); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc c"); + ujPtr->_finish(resultRows); //&&&uj flush and finish need to happen elsewhere, put it in qdisppool. + + + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc end"); + }; + + //&&&fileCollectFunc(); + + //&&&auto cmd = std::make_shared(fileCollectFunc); + auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); + exec->queueFileCollect(cmd); + + + /* &&&uj no need for this + proto::ResponseSummary responseSummary; + if (!(responseSummary.ParseFromArray(message, messageSize) && responseSummary.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); + } + */ + + + // 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; + +#if 0 //&&& + // It's possible jq and _jobQuery differ, so need to use jq. + if (job->isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, "QueryRequest::_processData job was cancelled."); + _errorFinish(true); + return false; + } + auto jq = std::dynamic_pointer_cast(job); + if (jq == nullptr) { + throw util::Bug(ERR_LOC, string(__func__) + " unexpected pointer type for job"); + } + 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; + } + + int messageSize = 0; + const char* message = GetMetadata(messageSize); + + LOGS(_log, LOG_LVL_DEBUG, __func__ << " _jobIdStr=" << _jobIdStr << ", messageSize=" << messageSize); + + proto::ResponseSummary responseSummary; + if (!(responseSummary.ParseFromArray(message, messageSize) && responseSummary.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); + } + uint32_t resultRows = 0; + if (!jq->getDescription()->respHandler()->flush(responseSummary, resultRows)) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " not flushOk"); + _flushError(jq); + return false; + } + _totalRows += resultRows; + + // 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(); + + // If the query meets the limit row complete complete criteria, it will start + // squashing superfluous results so the answer can be returned quickly. + executive->addResultRows(_totalRows); + executive->checkLimitRowComplete(); + + return true; +#endif // &&& } +json UberJob::_errorFinish(bool shouldCancel) { + json jsRet = {{"success", 0}, {"errortype", "dataproblem"}, {"note", ""}}; + /// &&&uj NEED CODE + /// - 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. + /// &&&uj see QueryRequest for some details + /// If shouldCancel is false, it may be possible to recover, so all + /// jobs that were in this query should marked NEED_RETRY so they + /// will be retried. + /// If shouldCancel is true, this function should call markComplete + /// for all jobs in the uberjob, with all jobs failed. + /// + /// In both case, the worker should delete the file as + /// this czar will not ask for it, so return a "success:0" json + /// message to the worker. + if (shouldCancel) { + jsRet = {{"success", 0}, {"errortype", "cancelling"}, {"note", ""}}; + } else { + ; + } + return jsRet; +} + +nlohmann::json UberJob::_finish(uint64_t resultRows) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_finish a"); + /// &&&uj NEED CODE + /// &&&uj see QueryRequest for some details + /// 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. + auto exec = _executive.lock(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, "UberJob::_finish executive is null qId=" << getQueryId() << " ujId=" << getJobId()); + return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + } + + bool const success = true; + callMarkCompleteFunc(success); + exec->addResultRows(resultRows); + exec->checkLimitRowComplete(); + + + + json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_finish end"); + return jsRet; +} + + std::ostream& UberJob::dumpOS(std::ostream& os) const { os << "(jobs sz=" << _jobs.size() << "("; for (auto const& job : _jobs) { diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 6952ca0835..d276e10dd5 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -27,11 +27,13 @@ #include "qmeta/types.h" #include "czar/CzarChunkMap.h" // Need nested class. &&&uj Make non-nested? #include "czar/CzarRegistry.h" // Need nested class. &&&uj Make non-nested? -#include "qdisp/Executive.h" +//&&&#include "qdisp/Executive.h" #include "qdisp/JobBase.h" // This header declarations -namespace lsst { namespace qserv { namespace qdisp { +namespace lsst::qserv::qdisp { + +class JobQuery; class QueryRequest; @@ -41,7 +43,7 @@ class UberJob : public JobBase { static uint32_t getMagicNumber() { return 93452; } - static Ptr create(Executive::Ptr const& executive, std::shared_ptr const& respHandler, + 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); @@ -53,11 +55,11 @@ class UberJob : public JobBase { static int getFirstIdNumber() { return 9'000'000; } - bool addJob(JobQuery* job); + bool addJob(std::shared_ptr const& job); bool runUberJob(); QueryId getQueryId() const override { return _queryId; } // TODO:UJ relocate to JobBase - int getIdInt() const override { return _uberJobId; } + UberJobId getJobId() const override { return _uberJobId; } // &&&uj change name 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; } @@ -66,10 +68,15 @@ class UberJob : public 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 { + /* &&& std::lock_guard lock(_qrMtx); _queryRequestPtr = qr; + } + */ + ; // Do nothing as QueryRequest is only needed for xrootd. } bool verifyPayload() const; @@ -87,23 +94,32 @@ class UberJob : public JobBase { //&&&uj czar::CzarChunkMap::WorkerChunksData::Ptr getWorkerData() { return _workerData; } + /// &&&uj doc + nlohmann::json importResultFile(std::string const& fileUrl, uint64_t rowCount, uint64_t fileSize); + std::ostream& dumpOS(std::ostream& os) const override; private: - UberJob(Executive::Ptr const& executive, std::shared_ptr const& respHandler, int queryId, + UberJob(std::shared_ptr const& executive, std::shared_ptr const& respHandler, int queryId, int uberJobId, qmeta::CzarId czarId, czar::CzarChunkMap::WorkerChunksData::Ptr const& workerData); - void _setup() { - JobBase::Ptr jbPtr = shared_from_this(); - _respHandler->setJobQuery(jbPtr); - } + /// Used to setup elements that can't be done in the constructor. + void _setup(); + + /// &&&uj doc + nlohmann::json _errorFinish(bool shouldCancel); + + /// &&&uj doc + nlohmann::json _finish(uint64_t resultRows); + - std::vector _jobs; // &&&uj should be a shared ptr ??? + std::vector> _jobs; //&&&uj + std::mutex _jobsMtx; ///< Protects _jobs std::atomic _started{false}; bool _inSsi = false; - JobStatus::Ptr _jobStatus; + std::shared_ptr _jobStatus; // &&&uj uber jobstatus probably needs to different than jobstatus. - std::shared_ptr _queryRequestPtr; + //&&& std::shared_ptr _queryRequestPtr; std::mutex _qrMtx; std::string _payload; ///< XrdSsi message to be sent to the _workerResource. //&&&uj remove when possible @@ -124,6 +140,6 @@ class UberJob : public JobBase { czar::CzarRegistry::WorkerContactInfo::Ptr _wContactInfo; }; -}}} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qdisp #endif // LSST_QSERV_QDISP_UBERJOB_H diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 00a713950f..af61bc01ad 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -61,6 +61,7 @@ #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" +#include "qdisp/UberJob.h" #include "qproc/DatabaseModels.h" #include "query/ColumnRef.h" #include "query/SelectStmt.h" @@ -217,13 +218,13 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { void InfileMerger::mergeCompleteFor(int jobId) { std::lock_guard resultSzLock(_mtxResultSizeMtx); - _totalResultSize += _perJobResultSize[jobId]; + _totalResultSize += _perJobResultSize[jobId]; //&&&uj this can probably be simplified } bool InfileMerger::merge(proto::ResponseSummary const& responseSummary, proto::ResponseData const& responseData, std::shared_ptr const& jq) { - int const jobId = responseSummary.jobid(); + JobId const jobId = responseSummary.jobid(); std::string queryIdJobStr = QueryIdHelper::makeIdStr(responseSummary.queryid(), jobId); if (!_queryIdStrSet) { _setQueryIdStr(QueryIdHelper::makeIdStr(responseSummary.queryid())); @@ -332,6 +333,123 @@ bool InfileMerger::merge(proto::ResponseSummary const& responseSummary, return ret; } +bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { + auto jq = uberJob; // &&& replace jq with uberJob + //&&&JobId const jobId = responseSummary.jobid(); + UberJobId const uJobId = uberJob->getJobId(); + //&&&std::string queryIdJobStr = QueryIdHelper::makeIdStr(responseSummary.queryid(), jobId); + std::string queryIdJobStr = uberJob->getIdStr(); + if (!_queryIdStrSet) { + //&&&_setQueryIdStr(QueryIdHelper::makeIdStr(responseSummary.queryid())); + _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 (jq->isQueryCancelled()) { + return true; + } + auto executive = jq->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(responseSummary.jobid(), responseSummary.attemptcount()); + 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 116aabaf1c..ef23ed1c11 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -55,6 +55,7 @@ class ResponseSummary; namespace qdisp { class JobQuery; class MessageStore; +class UberJob; } // namespace qdisp namespace qproc { class DatabaseModels; @@ -165,6 +166,10 @@ class InfileMerger { bool merge(proto::ResponseSummary const& responseSummary, proto::ResponseData const& responseData, std::shared_ptr const& jq); + /// &&&uj doc + + 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/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index f411df080c..ac26219bea 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -37,6 +37,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" @@ -47,6 +48,7 @@ #include "util/TimeUtils.h" #include "xrdsvc/StreamBuffer.h" + // LSST headers #include "lsst/log/Log.h" @@ -283,18 +285,19 @@ FileChannelShared::FileChannelShared(shared_ptr const& sendC } //&&&uj -FileChannelShared::Ptr FileChannelShared::create(UberJobId uberJobId, qmeta::CzarId czarId, +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(uberJobId, czarId, czarHostName, czarPort, workerId)); + return Ptr(new FileChannelShared(uberJob, czarId, czarHostName, czarPort, workerId)); } -FileChannelShared::FileChannelShared(UberJobId uberJobId, qmeta::CzarId czarId, string const& czarHostName, +FileChannelShared::FileChannelShared(std::shared_ptr const& uberJobData, qmeta::CzarId czarId, string const& czarHostName, int czarPort, string const& workerId) : _isUberJob(true), _sendChannel(nullptr), - _uberJobId(uberJobId), + _uberJobData(uberJobData), + _uberJobId(uberJobData->getUberJobId()), _czarId(czarId), _czarHostName(czarHostName), _czarPort(czarPort), @@ -573,6 +576,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) { + LOGS(_log, LOG_LVL_WARN, "&&& FileChannelShared::_sendResponse"); auto const queryId = task->getQueryId(); auto const jobId = task->getJobId(); auto const idStr(makeIdStr(queryId, jobId)); @@ -629,6 +633,8 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ 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) { // Send the message sent out-of-band within the SSI metadata. if (!_sendChannel->setMetadata(_responseBuf.data(), _responseBuf.size())) { @@ -650,8 +656,53 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ // &&&uj the http communications need to happen in a different thread, or this thread can be booted // from // &&&uj the scheduler so that it can just wait for a response. + 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(); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj ujId="<< _uberJobId << " httpFileUrl=" << httpFileUrl); + ujData->fileReadyResponse(httpFileUrl, _rowcount, _transmitsize); } return true; } +/* &&& +void FileChannelShared::_fileReadyResponse() { + json request = {{"version", http::MetaModule::version}, + {"workerid", _comInfoToCzar->foreman->chunkInventory()->id()}, + {"auth_key", authKey()}, + {"czar", czarName}, + {"czarid", czarId}, + {"queryid", queryId}, + {"uberjobid", uberJobId}}; + + auto const method = http::Method::POST; + vector const headers = {"Content-Type: application/json"}; + string const url = "http://" + czarHostName + ":" + to_string(czarPort) + "/queryjob-ready"; + string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + http::Client client(method, url, request.dump(), headers); + bool transmitSuccess = false; + try { + json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj response=" << response); + if (0 != response.at("success").get()) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj success"); + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE success=0"); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); + } + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, + __func__ << "&&&uj NEED CODE try again??? Let czar find out through polling worker status???"); + } else { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); + } +} +*/ + } // namespace lsst::qserv::wbase diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 716000cc3c..835a857cef 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -56,7 +56,14 @@ namespace lsst::qserv::util { class MultiError; } // namespace lsst::qserv::util +/* &&& +namespace lsst::qserv::wcontrol { +class Foreman; +} +*/ + 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 @@ -120,9 +127,16 @@ 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(UberJobId uberJobId, qmeta::CzarId czarId, std::string const& czarHostName, - int czarPort, std::string const& workerId); + int czarPort, std::string const& workerId, + ComInfoToCzar::Ptr const& comInfoToCzar); + */ + /// 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); //&&& delete all params except uberJob + FileChannelShared() = delete; FileChannelShared(FileChannelShared const&) = delete; @@ -176,7 +190,7 @@ class FileChannelShared { FileChannelShared(std::shared_ptr const& sendChannel, qmeta::CzarId czarId, std::string const& workerId); - FileChannelShared(UberJobId uberJobId, qmeta::CzarId czarId, std::string const& czarHostName, + FileChannelShared(std::shared_ptr const& uberJob, qmeta::CzarId czarId, std::string const& czarHostName, int czarPort, std::string const& workerId); /// @see wbase::SendChannel::kill @@ -237,16 +251,21 @@ class FileChannelShared { bool _sendResponse(std::lock_guard const& tMtxLock, std::shared_ptr const& task, bool cancelled, util::MultiError const& multiErr); + /// &&&uj doc + void _fileReadyResponse(); + 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. - UberJobId const _uberJobId; - qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). - std::string const _czarHostName; ///< Name of the czar host. - int const _czarPort; ///< port for the czar. - std::string const _workerId; ///< The unique identifier of the worker. + std::weak_ptr _uberJobData; ///< &&& doc + + UberJobId const _uberJobId; ///< &&& doc + qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). &&& delete + std::string const _czarHostName; ///< Name of the czar host. &&& delete + int const _czarPort; ///< port for the czar. &&& delete + std::string const _workerId; ///< The unique identifier of the worker. &&& delete // Allocatons/deletion of the data messages are managed by Google Protobuf Arena. std::unique_ptr _protobufArena; @@ -287,6 +306,8 @@ 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. + //&&&ComInfoToCzar::Ptr _comInfoToCzar; ///< &&&uj doc + 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. }; diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 2d58eb19c1..c2e5df7482 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -26,12 +26,23 @@ // Third party headers -// Qserv 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 "wcontrol/Foreman.h" +#include "wpublish/ChunkInventory.h" +#include "wpublish/QueriesAndChunks.h" + using namespace std; +using namespace nlohmann; namespace { @@ -41,13 +52,81 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wbase.UberJobData"); namespace lsst::qserv::wbase { -UberJobData::UberJobData(UberJobId uberJobId, qmeta::CzarId czarId, std::string czarHost, int czarPort, - uint64_t queryId, std::string const& workerId) +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) {} + _workerId(workerId), + _authKey(authKey), + _foreman(foreman) {} + +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::fileReadyResponse(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize) { + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse a httpFileUrl=" << httpFileUrl); + + 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}}; + + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse b"); + + auto const method = http::Method::POST; + vector const headers = {"Content-Type: application/json"}; + //&&&string const url = "http://" + _czarName + ":" + to_string(_czarPort) + "/queryjob-ready"; + string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-ready"; + string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse c"); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJobData::fileReadyResponse url=" << url << " request=" << request.dump()); + http::Client client(method, url, request.dump(), headers); + + int maxTries = 2; // &&& set from config + bool transmitSuccess = false; + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse d"); + for (int j=0; j()) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj success"); + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE success=0"); + } + } catch (exception const& ex) { + LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); + } + } + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse e"); + + if (!transmitSuccess) { + LOGS(_log, LOG_LVL_ERROR, + __func__ << "&&&uj NEED CODE Let czar find out through polling worker status???"); + } else { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); + } + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse end"); +} + + } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index a17ceba5c7..642f12b865 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -30,15 +30,24 @@ #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::wcontrol { +class Foreman; +} + namespace lsst::qserv::wbase { +class FileChannelShared; +class Task; + // &&&uj doc +/// This class tracks all Tasks associates with the UberJob and reports status to the czar. class UberJobData { public: using Ptr = std::shared_ptr; @@ -46,10 +55,13 @@ class UberJobData { UberJobData() = delete; UberJobData(UberJobData const&) = delete; - static Ptr create(UberJobId uberJobId, qmeta::CzarId czarId, std::string const& czarHost, int czarPort, - uint64_t queryId, std::string const& workerId) { - return Ptr(new UberJobData(uberJobId, czarId, czarHost, czarPort, queryId, workerId)); + 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)); } + // &&& doc + void setFileChannelShared(std::shared_ptr const& fileChannelShared); UberJobId getUberJobId() const { return _uberJobId; } qmeta::CzarId getCzarId() const { return _czarId; } @@ -58,16 +70,43 @@ class UberJobData { uint64_t getQueryId() const { return _queryId; } std::string getWorkerId() const { return _workerId; } + /// &&& doc + void addTasks(std::vector> const& tasks) { + _ujTasks.insert(_ujTasks.end(), tasks.begin(), tasks.end()); + } + + /// &&& doc + void fileReadyResponse(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize); + private: - UberJobData(UberJobId uberJobId, qmeta::CzarId czarId, std::string czarHost, int czarPort, - uint64_t queryId, std::string const& workerId); + 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; - uint64_t const _queryId; - std::string const& _workerId; //&&&uj should be able to get this from the worker in a reasonable way. + QueryId const _queryId; + std::string const _workerId; //&&&uj should be able to get this from the worker in a reasonable way. + std::string const _authKey; + + std::shared_ptr const _foreman; + + std::vector> _ujTasks; + std::shared_ptr _fileChannelShared; + + //&&&std::shared_ptr const foreman; + //&&& std::string const targetWorkerId; _workerId + //&&&std::string const czarName; + //&&&qmeta::CzarId const czarId; + //&&&std::string const czarHostName; _czarHost + //&&& int const czarPort; + //&&& uint64_t const queryId; + //&&&uint64_t const uberJobId; + + }; } // namespace lsst::qserv::wbase 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..b23865bb29 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); + /// &&& doc + 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 5b1889ab8d..4303fa6e2d 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -37,6 +37,7 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wsched.ChunkTasksQueue"); namespace lsst::qserv::wsched { +/* &&& /// Queue a Task with other tasks on the same chunk. void ChunkTasksQueue::queueTask(std::vector const& tasks) { std::lock_guard lg(_mapMx); @@ -65,6 +66,41 @@ void ChunkTasksQueue::queueTask(std::vector const& tasks) { iter->second->queTask(task); } } +*/ + +/// 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. 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. + std::pair ele(chunkId, std::make_shared(chunkId, _memMan)); + auto res = _chunkMap.insert(ele); // insert should fail if the key already exists. + LOGS(_log, LOG_LVL_DEBUG, " queueTask chunk=" << chunkId << " created=" << res.second); + iter = res.first; + } + } + ++_taskCount; + iter->second->queTask(task); + } +} /// Queue a Task with other tasks on the same chunk. void ChunkTasksQueue::queueTask(wbase::Task::Ptr const& task) { diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index 103c6751c7..dd182ddc07 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -224,29 +224,31 @@ 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 != t->getQueryId() || jid != 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; } } - t->setMemMan(_memMan); - 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/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 88ab12ae00..78b5727396 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -39,9 +39,11 @@ #include "mysql/MySqlUtils.h" #include "qmeta/types.h" // &&&uj #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" @@ -143,11 +145,19 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); - //&&&uj make UberJobData, FileChannelShared, and Tasks. + //&&&uj make UberJobData, UseQueryInfo entry, FileChannelShared, and Tasks. auto ujData = - wbase::UberJobData::create(ujId, czarId, czarHostName, czarPort, ujQueryId, targetWorkerId); + wbase::UberJobData::create(ujId, czarName, czarId, czarHostName, czarPort, ujQueryId, targetWorkerId, foreman(), authKey()); + LOGS(_log, LOG_LVL_WARN, "&&&uj (ujData != nullptr) = " << (ujData != nullptr)); + + // 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(ujId, czarId, czarHostName, czarPort, targetWorkerId); + wbase::FileChannelShared::create(ujData, czarId, czarHostName, czarPort, targetWorkerId); + ujData->setFileChannelShared(channelShared); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); for (auto const& job : ujJobs) { @@ -214,6 +224,13 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { 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_WARN, __func__ << "&&& Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); #if 0 /// &&&&&&&& // Now that the request is decoded (successfully or not), release the @@ -233,10 +250,10 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { // &&&uj temporary, send response back to czar saying file is ready. The file is not ready, but this // is just an initial comms test - _temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); + //&&&_temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); string note = string("qId=") + to_string(ujQueryId) + " ujId=" + to_string(ujId) + - "tasks in uberJob=" + to_string(channelShared->getTaskCount()); + " tasks in uberJob=" + to_string(channelShared->getTaskCount()); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC note=" << note); jsRet = {{"success", 1}, {"errortype", "none"}, {"note", note}}; @@ -247,9 +264,11 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { return jsRet; } +// &&&uj delete void HttpWorkerCzarModule::_temporaryRespFunc(string const& targetWorkerId, string const& czarName, qmeta::CzarId czarId, string const& czarHostName, int czarPort, uint64_t queryId, uint64_t uberJobId) { + json request = {{"version", http::MetaModule::version}, {"workerid", foreman()->chunkInventory()->id()}, {"auth_key", authKey()}, @@ -260,8 +279,9 @@ void HttpWorkerCzarModule::_temporaryRespFunc(string const& targetWorkerId, stri auto const method = http::Method::POST; vector const headers = {"Content-Type: application/json"}; - string const url = "http://" + czarHostName + ":" + to_string(czarPort) + "/queryjob-ready"; + string const url = "http://" + czarHostName + ":" + to_string(czarPort) + "/queryjob-error"; string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + LOGS(_log, LOG_LVL_WARN, "czarName=" << czarName << " czarHostName=" << czarHostName << " &&&uj HttpWorkerCzarModule::_temporaryRespFunc url=" << url << " request=" << request.dump()); http::Client client(method, url, request.dump(), headers); bool transmitSuccess = false; try { From 7f6d074f7211356d7acc72d3ff591eb66fd132cf Mon Sep 17 00:00:00 2001 From: John Gates Date: Fri, 21 Jun 2024 16:26:24 -0700 Subject: [PATCH 05/15] Added some error handling. --- src/ccontrol/MergingHandler.cc | 92 ++-- src/ccontrol/MergingHandler.h | 3 +- src/ccontrol/UserQuery.h | 6 +- src/ccontrol/UserQueryAsyncResult.cc | 12 +- src/ccontrol/UserQueryAsyncResult.h | 6 +- src/ccontrol/UserQueryDrop.cc | 4 +- src/ccontrol/UserQueryDrop.h | 4 +- src/ccontrol/UserQueryFactory.cc | 5 +- src/ccontrol/UserQueryFlushChunksCache.cc | 4 +- src/ccontrol/UserQueryFlushChunksCache.h | 4 +- src/ccontrol/UserQueryInvalid.h | 8 +- src/ccontrol/UserQueryProcessList.cc | 6 +- src/ccontrol/UserQueryProcessList.h | 4 +- src/ccontrol/UserQueryQservManager.cc | 4 +- src/ccontrol/UserQueryQservManager.h | 8 +- src/ccontrol/UserQuerySelect.cc | 514 +++++++++++++++------- src/ccontrol/UserQuerySelect.h | 32 +- src/ccontrol/UserQuerySelectCountStar.cc | 4 +- src/ccontrol/UserQuerySelectCountStar.h | 10 +- src/ccontrol/UserQuerySet.cc | 4 +- src/ccontrol/UserQuerySet.h | 4 +- src/czar/CzarRegistry.cc | 7 + src/czar/CzarRegistry.h | 1 + src/czar/HttpCzarWorkerModule.cc | 74 +++- src/czar/HttpCzarWorkerModule.h | 2 + src/czar/MessageTable.cc | 4 +- src/global/intTypes.h | 2 +- src/qdisp/CMakeLists.txt | 2 - src/qdisp/Executive.cc | 284 ++++++++++-- src/qdisp/Executive.h | 92 +++- src/qdisp/JobBase.h | 11 +- src/qdisp/JobQuery.cc | 34 +- src/qdisp/JobQuery.h | 63 ++- src/qdisp/QueryRequest.cc | 18 +- src/qdisp/QueryRequest.h | 2 +- src/qdisp/ResponseHandler.h | 3 +- src/qdisp/UberJob.cc | 462 ++++++++++--------- src/qdisp/UberJob.h | 72 ++- src/qdisp/testQDisp.cc | 20 +- src/qhttp/Server.cc | 27 ++ src/qmeta/CMakeLists.txt | 3 +- src/{qdisp => qmeta}/JobStatus.cc | 40 +- src/{qdisp => qmeta}/JobStatus.h | 35 +- src/{qdisp => qmeta}/MessageStore.cc | 15 +- src/{qdisp => qmeta}/MessageStore.h | 18 +- src/qmeta/QMeta.h | 5 +- src/qmeta/QMetaMysql.cc | 18 +- src/qmeta/QMetaMysql.h | 6 +- src/qmeta/testQMeta.cc | 1 + src/qproc/TaskMsgFactory.cc | 4 +- src/rproc/InfileMerger.cc | 5 +- src/rproc/InfileMerger.h | 4 +- src/wbase/FileChannelShared.cc | 129 +++--- src/wbase/FileChannelShared.h | 13 +- src/wbase/Task.cc | 4 +- src/wbase/UberJobData.cc | 115 ++++- src/wbase/UberJobData.h | 36 +- src/wbase/UserQueryInfo.h | 2 +- src/wsched/ChunkTasksQueue.cc | 2 +- src/wsched/ScanScheduler.cc | 1 - src/xrdsvc/HttpWorkerCzarModule.cc | 10 +- src/xrdsvc/HttpWorkerCzarModule.h | 2 + 62 files changed, 1628 insertions(+), 758 deletions(-) rename src/{qdisp => qmeta}/JobStatus.cc (70%) rename src/{qdisp => qmeta}/JobStatus.h (77%) rename src/{qdisp => qmeta}/MessageStore.cc (87%) rename src/{qdisp => qmeta}/MessageStore.h (90%) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 7d108ab30e..c0efa698bc 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -361,7 +361,7 @@ bool readHttpFileAndMerge(string const& httpUrl, to_string(offset - msgSizeBytes) + ", file: " + httpUrl); } } catch (exception const& ex) { - LOGS(_log, LOG_LVL_ERROR, ex.what()); + LOGS(_log, LOG_LVL_ERROR, string(__func__) + " " + ex.what()); success = false; } @@ -376,11 +376,12 @@ bool readHttpFileAndMerge(string const& httpUrl, return success; } -std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, - function const& messageIsReady, - shared_ptr const& httpConnPool) { +std::tuple readHttpFileAndMergeHttp( + lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, + function const& messageIsReady, + shared_ptr const& httpConnPool) { LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp a"); - string const context = "MergingHandler::" + string(__func__) + " "; + string const context = "MergingHandler::" + string(__func__) + " " + " qid=" + uberJob->getIdStr() + " "; LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); @@ -410,7 +411,9 @@ std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr uint32_t msgSizeBytes = 0; bool success = true; bool mergeSuccess = true; - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp b"); + LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp b"); + int headerCount = 0; // &&& del + uint64_t totalBytesRead = 0; /// &&& del try { string const noClientData; vector const noClientHeaders; @@ -420,19 +423,25 @@ std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr clientConfig.tcpKeepAlive = true; clientConfig.tcpKeepIdle = 5; // the default is 60 sec clientConfig.tcpKeepIntvl = 5; // the default is 60 sec - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp c"); + LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp c"); http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, httpConnPool); - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp d"); + LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp d"); reader.read([&](char const* inBuf, size_t inBufSize) { + // LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp d1 reader.read + // ok"); // 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_WARN, "&&& readHttpFileAndMergeHttp e"); + LOGS(_log, LOG_LVL_WARN, + context + "&&& readHttpFileAndMergeHttp e next=" << (uint64_t)next << " end=" + << (uint64_t)end << " last=" << last); while ((next < end) && !last) { - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e1"); + LOGS(_log, LOG_LVL_WARN, + context + "&&& readHttpFileAndMergeHttp e1 next=" << (uint64_t)next << " end=" + << (uint64_t)end << " last=" << last); if (msgSizeBytes == 0) { LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e2"); // Continue or finish reading the frame header. @@ -443,18 +452,20 @@ std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr offset += bytes2read; msgSizeBufNext += bytes2read; if (msgSizeBufNext == sizeof(uint32_t)) { - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e3"); + ++headerCount; + LOGS(_log, LOG_LVL_WARN, + context + "&&& readHttpFileAndMergeHttp e3 &&& headerCount=" << 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(context + "message size is 0 at offset " + + 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(context + "message size " + to_string(msgSizeBytes) + - " at offset " + to_string(offset - sizeof(uint32_t)) + + 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); @@ -494,32 +505,47 @@ std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr // Parse and evaluate the message. LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e6"); mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e7"); + LOGS(_log, LOG_LVL_WARN, + context + "&&& readHttpFileAndMergeHttp e7 next=" + << (uint64_t)next << " end=" << (uint64_t)end << " last=" << last + << " success=" << success); + totalBytesRead += msgSizeBytes; + LOGS(_log, LOG_LVL_WARN, + context + "&&& readHttpFileAndMergeHttp e7 headerCount=" + << headerCount << " msgSizeBytes=" << msgSizeBytes + << " totalBytesRead=" << totalBytesRead); if (!mergeSuccess) { success = false; - throw runtime_error(context + "message processing failed at offset " + + 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; LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e8"); + } else { + LOGS(_log, LOG_LVL_WARN, + "&&&uj headerCount=" << headerCount + << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); } } } }); - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e9"); + LOGS(_log, LOG_LVL_WARN, + context + "&&& readHttpFileAndMergeHttp e9 headerCount=" + << headerCount << " msgSizeBytes=" << msgSizeBytes + << " totalBytesRead=" << totalBytesRead); if (msgSizeBufNext != 0) { - throw runtime_error(context + "short read of the message header at offset " + + throw runtime_error("short read of the message header at offset " + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); } LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e10"); if (msgBufNext != 0) { - throw runtime_error(context + "short read of the message body at offset " + + throw runtime_error("short read of the message body at offset " + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); } LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e11"); - } catch (exception const& ex) { // &&&uj anything being caught here besides runtime_error? - LOGS(_log, LOG_LVL_ERROR, ex.what()); + } catch (exception const& ex) { // &&&uj anything being caught here besides runtime_error? + LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); success = false; } @@ -536,11 +562,11 @@ std::tuple readHttpFileAndMergeHttp(lsst::qserv::qdisp::UberJob::Ptr } // 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_WARN, "&&& readHttpFileAndMergeHttp end succes=" << success << " mergeSuccess=" << mergeSuccess); + LOGS(_log, LOG_LVL_WARN, + "&&& readHttpFileAndMergeHttp end succes=" << success << " mergeSuccess=" << mergeSuccess); return {success, mergeSuccess}; } - } // namespace namespace lsst::qserv::ccontrol { @@ -676,7 +702,8 @@ bool MergingHandler::_merge(proto::ResponseSummary const& responseSummary, return success; } -bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, proto::ResponseData const& responseData) { +bool MergingHandler::_mergeHttp(shared_ptr const& uberJob, + proto::ResponseData const& responseData) { if (_flushed) { throw util::Bug(ERR_LOC, "already flushed"); } @@ -695,7 +722,8 @@ void MergingHandler::_setError(int code, std::string const& msg) { _error = Error(code, msg); } -tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expectedRows, uint64_t& resultRows) { +tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) { bool success = false; bool shouldCancel = false; /// &&&uj NEED CODE @@ -706,14 +734,13 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe 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 + return {success, shouldCancel}; // both should still be false } //&&& auto const jobQuery = std::dynamic_pointer_cast(jobBase); auto const uberJob = std::dynamic_pointer_cast(jobBase); LOGS(_log, LOG_LVL_TRACE, - "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() - << " fileUrl=" << fileUrl); + "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); /* &&& errors will be handled by MergingHandler::flushHttpError() if (responseSummary.errorcode() != 0 || !responseSummary.errormsg().empty()) { @@ -766,8 +793,11 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe throw runtime_error("MergingHandler::flush ** message deserialization failed **"); }; - tie(success, shouldCancel) = ::readHttpFileAndMergeHttp(uberJob, fileUrl, dataMergerHttp, - MergingHandler::_getHttpConnPool()); + tie(success, shouldCancel) = + ::readHttpFileAndMergeHttp(uberJob, fileUrl, dataMergerHttp, MergingHandler::_getHttpConnPool()); + + LOGS(_log, LOG_LVL_WARN, + "&&& MergingHandler::flushHttp success=" << success << " shouldCancel=" << shouldCancel); if (success) { _infileMerger->mergeCompleteFor(uberJob->getJobId()); @@ -775,6 +805,4 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe return {success, shouldCancel}; } - - } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index c8f4c3c538..ca3a4b7f03 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -80,7 +80,8 @@ class MergingHandler : public qdisp::ResponseHandler { /// @return shouldCancel - if success was false, this being true indicates there /// was an unrecoverable error in table writing and the query /// should be cancelled. - std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, uint64_t& resultRows) override; + std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, + uint64_t& resultRows) override; /// &&&uj doc bool flushHttpError(); 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 9f11c46e50..a3edbbcc2c 100644 --- a/src/ccontrol/UserQueryAsyncResult.cc +++ b/src/ccontrol/UserQueryAsyncResult.cc @@ -32,9 +32,9 @@ // Qserv headers #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/SqlResults.h" @@ -53,7 +53,7 @@ UserQueryAsyncResult::UserQueryAsyncResult(QueryId queryId, qmeta::CzarId qMetaC _qMetaCzarId(qMetaCzarId), _qMeta(qMeta), _resultDbConn(resultDbConn), - _messageStore(std::make_shared()) { + _messageStore(std::make_shared()) { LOGS(_log, LOG_LVL_DEBUG, "UserQueryAsyncResult: QID=" << queryId); // get query info from QMeta @@ -149,8 +149,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()); @@ -184,7 +184,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 9b2340211d..d65e8a76da 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; } @@ -90,7 +90,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() @@ -113,7 +113,7 @@ class UserQueryAsyncResult : public UserQuery { std::shared_ptr _qMeta; sql::SqlConnection* _resultDbConn; qmeta::QInfo _qInfo; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; QueryState _qState = UNKNOWN; }; diff --git a/src/ccontrol/UserQueryDrop.cc b/src/ccontrol/UserQueryDrop.cc index 5e291800ff..842f68c250 100644 --- a/src/ccontrol/UserQueryDrop.cc +++ b/src/ccontrol/UserQueryDrop.cc @@ -33,8 +33,8 @@ // Qserv headers #include "css/CssAccess.h" #include "css/CssError.h" -#include "qdisp/MessageStore.h" #include "qmeta/Exceptions.h" +#include "qmeta/MessageStore.h" #include "qmeta/QMeta.h" #include "sql/SqlConnection.h" #include "sql/SqlErrorObject.h" @@ -57,7 +57,7 @@ UserQueryDrop::UserQueryDrop(std::shared_ptr const& css, std::st _queryMetadata(queryMetadata), _qMetaCzarId(qMetaCzarId), _qState(UNKNOWN), - _messageStore(std::make_shared()), + _messageStore(std::make_shared()), _sessionId(0) {} std::string UserQueryDrop::getError() const { return std::string(); } diff --git a/src/ccontrol/UserQueryDrop.h b/src/ccontrol/UserQueryDrop.h index 73cc6b6b61..450f06fab8 100644 --- a/src/ccontrol/UserQueryDrop.h +++ b/src/ccontrol/UserQueryDrop.h @@ -88,7 +88,7 @@ class UserQueryDrop : 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: /// Check the status of item to be dropped @@ -101,7 +101,7 @@ class UserQueryDrop : public UserQuery { std::shared_ptr _queryMetadata; qmeta::CzarId const _qMetaCzarId; ///< Czar ID in QMeta database QueryState _qState; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; int _sessionId; ///< External reference number }; diff --git a/src/ccontrol/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index 26b808822f..d3fe5bce5e 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -56,7 +56,7 @@ #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/QStatusMysql.h" @@ -306,7 +306,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) { @@ -329,6 +329,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/UserQueryFlushChunksCache.cc b/src/ccontrol/UserQueryFlushChunksCache.cc index 2c808c139f..47037d127a 100644 --- a/src/ccontrol/UserQueryFlushChunksCache.cc +++ b/src/ccontrol/UserQueryFlushChunksCache.cc @@ -31,7 +31,7 @@ // Qserv headers #include "css/CssAccess.h" #include "css/EmptyChunks.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "sql/SqlConnection.h" #include "sql/SqlErrorObject.h" @@ -49,7 +49,7 @@ UserQueryFlushChunksCache::UserQueryFlushChunksCache(std::shared_ptr()) {} + _messageStore(std::make_shared()) {} std::string UserQueryFlushChunksCache::getError() const { return std::string(); } diff --git a/src/ccontrol/UserQueryFlushChunksCache.h b/src/ccontrol/UserQueryFlushChunksCache.h index 74054aaaa3..fe4e913c0b 100644 --- a/src/ccontrol/UserQueryFlushChunksCache.h +++ b/src/ccontrol/UserQueryFlushChunksCache.h @@ -82,7 +82,7 @@ class UserQueryFlushChunksCache : public UserQuery { virtual void discard() override; // Delegate objects - virtual std::shared_ptr getMessageStore() override { return _messageStore; } + virtual std::shared_ptr getMessageStore() override { return _messageStore; } protected: private: @@ -90,7 +90,7 @@ class UserQueryFlushChunksCache : public UserQuery { std::string const _dbName; sql::SqlConnection* _resultDbConn; QueryState _qState; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; }; } // namespace lsst::qserv::ccontrol 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 85a626ecaf..686c8d0bd4 100644 --- a/src/ccontrol/UserQueryProcessList.cc +++ b/src/ccontrol/UserQueryProcessList.cc @@ -35,7 +35,7 @@ // Qserv headers #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 @@ UserQueryProcessList::UserQueryProcessList(std::shared_ptr co : _resultDbConn(resultDbConn), _qMetaSelect(qMetaSelect), _qMetaCzarId(qMetaCzarId), - _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 @@ -104,7 +104,7 @@ UserQueryProcessList::UserQueryProcessList(bool full, sql::SqlConnection* result : _resultDbConn(resultDbConn), _qMetaSelect(qMetaSelect), _qMetaCzarId(qMetaCzarId), - _messageStore(std::make_shared()), + _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), _resultDb(resultDb) { // use ShowProcessList view with completion statistics. diff --git a/src/ccontrol/UserQueryProcessList.h b/src/ccontrol/UserQueryProcessList.h index 260d3a0adc..8dd421d401 100644 --- a/src/ccontrol/UserQueryProcessList.h +++ b/src/ccontrol/UserQueryProcessList.h @@ -103,7 +103,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; } @@ -122,7 +122,7 @@ class UserQueryProcessList : public UserQuery { std::shared_ptr _qMetaSelect; qmeta::CzarId const _qMetaCzarId; ///< 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/UserQueryQservManager.cc b/src/ccontrol/UserQueryQservManager.cc index 7d8065d63b..5f132c5802 100644 --- a/src/ccontrol/UserQueryQservManager.cc +++ b/src/ccontrol/UserQueryQservManager.cc @@ -36,7 +36,7 @@ // Qserv headers #include "cconfig/CzarConfig.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "sql/SqlBulkInsert.h" #include "sql/SqlConnection.h" #include "sql/SqlConnectionFactory.h" @@ -56,7 +56,7 @@ UserQueryQservManager::UserQueryQservManager(shared_ptr cons string const& value) : _value(value), _resultTableName("qserv_manager_" + queryResources->userQueryId), - _messageStore(make_shared()), + _messageStore(make_shared()), _resultDb(queryResources->resultDb) {} void UserQueryQservManager::submit() { diff --git a/src/ccontrol/UserQueryQservManager.h b/src/ccontrol/UserQueryQservManager.h index fee3e56248..0c73a9f659 100644 --- a/src/ccontrol/UserQueryQservManager.h +++ b/src/ccontrol/UserQueryQservManager.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,7 +74,7 @@ class UserQueryQservManager : public UserQuery { void discard() override {} // Delegate objects - std::shared_ptr getMessageStore() override { return _messageStore; } + std::shared_ptr getMessageStore() override { return _messageStore; } std::string getResultLocation() const override { return "table:" + _resultTableName; } @@ -84,7 +84,7 @@ class UserQueryQservManager : public UserQuery { private: std::string const _value; std::string _resultTableName; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; QueryState _qState{UNKNOWN}; std::string _resultDb; }; diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 004179554f..1e804a6cb7 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -88,8 +88,7 @@ #include "proto/worker.pb.h" #include "proto/ProtoImporter.h" #include "qdisp/Executive.h" -#include "qdisp/MessageStore.h" - +#include "qmeta/MessageStore.h" #include "qmeta/QMeta.h" #include "qmeta/Exceptions.h" #include "qproc/geomAdapter.h" @@ -135,7 +134,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, @@ -340,7 +339,8 @@ void UserQuerySelect::submit() { //&&&uj assert(_infileMerger); auto taskMsgFactory = std::make_shared(); - TmpTableName ttn(_qMetaQueryId, _qSession->getOriginal()); + _ttn = std::make_shared(_qMetaQueryId, _qSession->getOriginal()); + //&&&_executive->setTmpTableNameGenerator(ttn); std::vector chunks; std::mutex chunksMtx; JobId sequence = 0; @@ -381,7 +381,7 @@ void UserQuerySelect::submit() { //&&&uj cs = _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec); chunks.push_back(cs->chunkId); } - std::string chunkResultName = ttn.make(cs->chunkId); + std::string chunkResultName = _ttn->make(cs->chunkId); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d2"); // This should only need to be set once as all jobs should have the same database name. @@ -428,176 +428,378 @@ void UserQuerySelect::submit() { //&&&uj ++sequence; } + /// &&& ******************************************************** + /// &&&uj at this point the executive has a map of all jobs with the chunkIds as the key. + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e"); //&&&if (uberJobsEnabled || true) { +#if 1 // &&& if (uberJobsEnabled) { LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); - vector uberJobs; - - auto czarPtr = czar::Czar::getCzar(); - auto czChunkMap = czarPtr->getCzarChunkMap(); - auto czRegistry = czarPtr->getCzarRegistry(); - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1a"); - auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj - - // Make a map of all jobs in the executive. - // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can - // be found for all databases in the query - qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1b"); - - // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use - // the same databases) Use this to check for conflicts - - // assign jobs to uberJobs - int maxChunksPerUber = 1; // &&&uj maybe put in config??? or set on command line?? + _maxChunksPerUberJob = 2; // &&&uj maybe put in config??? or set on command line?? // &&&uj Different queries may benefit from different values // &&&uj Such as LIMIT=1 may work best with this at 1, where // &&&uj 100 would be better for others. - // keep cycling through workers until no more chunks to place. - - // TODO:UJ &&&uj So UberJobIds don't conflict with chunk numbers or jobIds, start at a large number. - // This could use some refinement. - int uberJobId = qdisp::UberJob::getFirstIdNumber(); - - // &&&uj - // - create a map of UberJobs key=, val=> - // - for chunkId in `chunksInQuery` - // - 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, append a new uberjob to the vect - // - 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; - - // chunksInQuery 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. - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c"); - for (auto const& [chunkId, jqPtr] : chunksInQuery) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c1"); - auto iter = chunkMapPtr->find(chunkId); - if (iter == chunkMapPtr->end()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c2"); - missingChunks.push_back(chunkId); - break; - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c3"); - czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; - auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c4"); - if (targetWorker == nullptr) { - LOGS(_log, LOG_LVL_ERROR, "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, "Alternate worker found for chunk=" << chunkData->dump()); - } - } - if (!found) { - // &&&uj If enough workers are down, there will be a chunk that cannot be found. - // the correct course of action is probably to check the Registry, and - // after so many attempts, cancel the user query with a - // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount - // in the Job or JobDescription could be used for this. - LOGS(_log, LOG_LVL_ERROR, - "No primary or alternate worker found for chunk=" << chunkData->dump()); - throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + - " Crashing the program here for this reason is not " - "appropriate. &&& NEEDS CODE"); + //&&&_executive->buildAndSendUberJobs(maxChunksPerUber); + buildAndSendUberJobs(); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e2"); +#else // &&& + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); + vector uberJobs; + + auto czarPtr = czar::Czar::getCzar(); + auto czChunkMap = czarPtr->getCzarChunkMap(); + auto czRegistry = czarPtr->getCzarRegistry(); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1a"); + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj + + // Make a map of all jobs in the executive. + // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can + // be found for all databases in the query + qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1b"); + + // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use + // the same databases) Use this to check for conflicts + + // assign jobs to uberJobs + int maxChunksPerUber = 3; // &&&uj maybe put in config??? or set on command line?? + // &&&uj Different queries may benefit from different values + // &&&uj Such as LIMIT=1 may work best with this at 1, where + // &&&uj 100 would be better for others. + // keep cycling through workers until no more chunks to place. + + // TODO:UJ &&&uj So UberJobIds don't conflict with chunk numbers or jobIds, start at a large number. + // This could use some refinement. + int uberJobId = qdisp::UberJob::getFirstIdNumber(); + + // &&&uj + // - create a map of UberJobs key=, val=> + // - for chunkId in `chunksInQuery` + // - 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, append a new uberjob to the vect + // - 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; + + // chunksInQuery 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. + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c"); + for (auto const& [chunkId, jqPtr] : chunksInQuery) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c1"); + auto iter = chunkMapPtr->find(chunkId); + if (iter == chunkMapPtr->end()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c2"); + missingChunks.push_back(chunkId); + break; + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c3"); + czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; + auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c4"); + if (targetWorker == nullptr) { + LOGS(_log, LOG_LVL_ERROR, "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, "Alternate worker found for chunk=" << chunkData->dump()); } } - // 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); + if (!found) { + // &&&uj If enough workers are down, there will be a chunk that cannot be found. + // the correct course of action is probably to check the Registry, and + // after so many attempts, cancel the user query with a + // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount + // in the Job or JobDescription could be used for this. + LOGS(_log, LOG_LVL_ERROR, + "No primary or alternate worker found for chunk=" << chunkData->dump()); + throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + + " Crashing the program here for this reason is not " + "appropriate. &&& NEEDS CODE"); } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c6"); - ujVect.back()->addJob(jqPtr); } + // 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); + } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c7"); - if (!missingChunks.empty()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c8"); - string errStr = string(__func__) + " a worker could not be found for these chunks "; - for (auto const& chk : missingChunks) { - errStr += to_string(chk) + ","; - } - LOGS(_log, LOG_LVL_ERROR, errStr); - throw util::Bug( - ERR_LOC, - errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c7"); + if (!missingChunks.empty()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c8"); + string errStr = string(__func__) + " a worker could not be found for these chunks "; + for (auto const& chk : missingChunks) { + errStr += to_string(chk) + ","; } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); - - //&&&uj - // Add worker contact info to UberJobs. - auto const wContactMap = czRegistry->getWorkerContactMap(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f"); - for (auto const& [wIdKey, ujVect] : workerJobMap) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f1"); - auto iter = wContactMap->find(wIdKey); - if (iter == wContactMap->end()) { - // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. - throw util::Bug(ERR_LOC, string(" &&&uj NEED CODE, no contact information for ") + wIdKey); - } - auto const& wContactInfo = iter->second; - for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f2"); - ujPtr->setWorkerContactInfo(wContactInfo); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f3"); - _executive->addUberJobs(ujVect); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f4"); - for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f5"); - //&&&uj IMPORTANT - //&&&uj This just sends the test message to the worker, but the - // worker only parses it and sends a message back. The - // worker does not create and run tasks at this point. - // The call to runUberJob here should be replaced by a call - // to startUberJob that puts the call to runUberJob into - // the a priority queue command. - _executive->runUberJob(ujPtr); - } + LOGS(_log, LOG_LVL_ERROR, errStr); + throw util::Bug( + ERR_LOC, + errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); + + //&&&uj + // Add worker contact info to UberJobs. + auto const wContactMap = czRegistry->getWorkerContactMap(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f"); + for (auto const& [wIdKey, ujVect] : workerJobMap) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f1"); + auto iter = wContactMap->find(wIdKey); + if (iter == wContactMap->end()) { + // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. + throw util::Bug(ERR_LOC, string(" &&&uj NEED CODE, no contact information for ") + wIdKey); } + auto const& wContactInfo = iter->second; + for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f2"); + ujPtr->setWorkerContactInfo(wContactInfo); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f3"); + _executive->addUberJobs(ujVect); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f4"); + for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f5"); + //&&&uj IMPORTANT + //&&&uj This just sends the test message to the worker, but the + // worker only parses it and sends a message back. The + // worker does not create and run tasks at this point. + // The call to runUberJob here should be replaced by a call + // to startUberJob that puts the call to runUberJob into + // the a priority queue command. + _executive->runUberJob(ujPtr); + } + } +#endif // &&& + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj + LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); + _executive->waitForAllJobsToStart(); - LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); - _executive->waitForAllJobsToStart(); + // we only care about per-chunk info for ASYNC queries + if (_async) { + std::lock_guard lock(chunksMtx); + _qMetaAddChunks(chunks); + } - // we only care about per-chunk info for ASYNC queries - if (_async) { - std::lock_guard lock(chunksMtx); - _qMetaAddChunks(chunks); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew end"); +} + +void UserQuerySelect::buildAndSendUberJobs() { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs a"); + string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); + LOGS(_log, LOG_LVL_INFO, funcN << " start"); + lock_guard fcLock(_buildUberJobMtx); + bool const clearFlag = false; + _executive->setFlagFailedUberJob(clearFlag); + LOGS(_log, LOG_LVL_WARN, + "&&& UserQuerySelect::buildAndSendUberJobs totalJobs=" << _executive->getTotalJobs()); + + vector uberJobs; + + auto czarPtr = czar::Czar::getCzar(); + auto czChunkMap = czarPtr->getCzarChunkMap(); + auto czRegistry = czarPtr->getCzarRegistry(); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs b"); + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj + + // Make a map of all jobs in the executive. + // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can + // be found for all databases in the query + /* &&& instead of destroying the only copy, going to make a copy that only contains + * unassigned jobs. + qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); + if (_chunkToJobMapInvalid.exchange(true)) { + throw util::Bug(ERR_LOC, "getChunkJobMapInvalidate called when map already invalid"); } + return _chunkToJobMap; + + */ + qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs c"); + + // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use + // the same databases) Use this to check for conflicts + + /* &&& + // assign jobs to uberJobs + int maxChunksPerUber = 3; // &&&uj maybe put in config??? or set on command line?? + // &&&uj Different queries may benefit from different values + // &&&uj Such as LIMIT=1 may work best with this at 1, where + // &&&uj 100 would be better for others. + */ + // keep cycling through workers until no more chunks to place. + + // TODO:UJ &&&uj Once everything is an UberJob, it can start with 1 or 0. + // int _uberJobId = qdisp::UberJob::getFirstIdNumber(); + + // &&&uj + // - 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. + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d"); + for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d1"); + auto iter = chunkMapPtr->find(chunkId); + if (iter == chunkMapPtr->end()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d1a"); + missingChunks.push_back(chunkId); + break; + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d2"); + ; + czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; + auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d3"); + if (targetWorker == + nullptr) { //&&&uj if (targetWorker == nullptr || this worker already tried for this chunk) { + 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) { + // &&&uj If too many workers are down, there will be a chunk that cannot be found. + // the correct course of action is probably to check the Registry, and + // after so many attempts, cancel the user query with a + // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount + // in the Job or JobDescription could be used for this. + LOGS(_log, LOG_LVL_ERROR, + funcN << " No primary or alternate worker found for chunk=" << chunkData->dump()); + throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + + " Crashing the program here for this reason is not " + "appropriate. &&& NEEDS CODE"); + } + } + // Add this job to the appropriate UberJob, making the UberJob if needed. + string workerId = targetWorker->getWorkerId(); + auto& ujVect = workerJobMap[workerId]; + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d4 ujVect.sz=" << ujVect.size()); + if (ujVect.empty() || ujVect.back()->getJobCount() >= _maxChunksPerUberJob) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d4a"); + 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(); + LOGS(_log, LOG_LVL_WARN, + "&&& UserQuerySelect::buildAndSendUberJobs d4b ujVectBack{" + << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() << "}"); + ujVectBack->addJob(jqPtr); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d5 ujVect.sz=" << ujVect.size()); + LOGS(_log, LOG_LVL_WARN, + "&&& UserQuerySelect::buildAndSendUberJobs d5a ujVectBack{" + << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() << "}"); } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew end"); + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d6"); + if (!missingChunks.empty()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d6a"); + string errStr = funcN + " a worker could not be found for these chunks "; + for (auto const& chk : missingChunks) { + errStr += to_string(chk) + ","; + } + LOGS(_log, LOG_LVL_ERROR, errStr); + throw util::Bug( + ERR_LOC, + errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs e"); + + //&&&uj + // Add worker contact info to UberJobs. + auto const wContactMap = czRegistry->getWorkerContactMap(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f"); + LOGS(_log, LOG_LVL_WARN, + "&&& UserQuerySelect::buildAndSendUberJobs f" << _executive->dumpUberJobCounts()); + for (auto const& [wIdKey, ujVect] : workerJobMap) { + LOGS(_log, LOG_LVL_WARN, + "&&& UserQuerySelect::buildAndSendUberJobs f1 wId=" << wIdKey << " ujVect.sz=" << ujVect.size()); + auto iter = wContactMap->find(wIdKey); + if (iter == wContactMap->end()) { + // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. + throw util::Bug(ERR_LOC, funcN + " &&&uj NEED CODE, no contact information for " + wIdKey); + } + auto const& wContactInfo = iter->second; + for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, + ujPtr->getIdStr() << " " << wContactInfo->dump() + << " &&& UserQuerySelect::buildAndSendUberJobs f2"); + ujPtr->setWorkerContactInfo(wContactInfo); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f3"); + _executive->addUberJobs(ujVect); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f4"); + for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f5"); + _executive->runUberJob(ujPtr); + } + } + LOGS(_log, LOG_LVL_WARN, + "&&& UserQuerySelect::buildAndSendUberJobs g" << _executive->dumpUberJobCounts()); } /// Block until a submit()'ed query completes. diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index 70b7d87a89..a69e07b1be 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -32,6 +32,7 @@ */ // System headers +#include #include #include #include @@ -50,13 +51,13 @@ // Forward declarations namespace lsst::qserv::qdisp { class Executive; -class MessageStore; class QdispPool; } // namespace lsst::qserv::qdisp namespace lsst::qserv::qmeta { +class MessageStore; class QMeta; -} +} // namespace lsst::qserv::qmeta namespace lsst::qserv::qproc { class DatabaseModels; @@ -80,11 +81,13 @@ class SemaMgr; 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, @@ -124,7 +127,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; } @@ -151,6 +154,9 @@ class UserQuerySelect : public UserQuery { /// save the result query in the query metadata void saveResultQuery(); + /// &&&uj doc + void buildAndSendUberJobs(); + private: /// @return ORDER BY part of SELECT statement that gets executed by the proxy std::string _getResultOrderBy() const; @@ -169,7 +175,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; @@ -185,11 +191,17 @@ class UserQuerySelect : public UserQuery { std::string _queryIdStr{QueryIdHelper::makeIdStr(0, true)}; bool _killed{false}; std::mutex _killMutex; - 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??) - bool _async; ///< true for async query + 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??) + bool _async; ///< true for async query + int _maxChunksPerUberJob = 1; ///< &&&uj + std::atomic _uberJobIdSeq{900'000}; ///< &&&uj can probably start at 1 + std::shared_ptr _ttn; ///< Temporary table name generator. + + /// &&&uj Only one thread should run buildAndSendUberJobs() at a time + std::mutex _buildUberJobMtx; }; } // namespace lsst::qserv::ccontrol diff --git a/src/ccontrol/UserQuerySelectCountStar.cc b/src/ccontrol/UserQuerySelectCountStar.cc index d796d810ce..d15ea9639d 100644 --- a/src/ccontrol/UserQuerySelectCountStar.cc +++ b/src/ccontrol/UserQuerySelectCountStar.cc @@ -29,7 +29,7 @@ // Qserv headers #include "ccontrol/UserQueryError.h" #include "ccontrol/UserQueryType.h" -#include "qdisp/MessageStore.h" +#include "qmeta/MessageStore.h" #include "qmeta/QInfo.h" #include "qmeta/QMetaSelect.h" #include "query/SelectStmt.h" @@ -62,7 +62,7 @@ UserQuerySelectCountStar::UserQuerySelectCountStar(std::string query, : _resultDbConn(resultDbConn), _qMetaSelect(qMetaSelect), _queryMetadata(queryMetadata), - _messageStore(std::make_shared()), + _messageStore(std::make_shared()), _resultTableName(::g_nextResultTableId(userQueryId)), _userQueryId(userQueryId), _rowsTable(rowsTable), diff --git a/src/ccontrol/UserQuerySelectCountStar.h b/src/ccontrol/UserQuerySelectCountStar.h index 46e5448e7e..681150f842 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; } @@ -82,7 +80,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() @@ -117,7 +115,7 @@ class UserQuerySelectCountStar : public UserQuery { std::shared_ptr _resultDbConn; std::shared_ptr _qMetaSelect; std::shared_ptr const& _queryMetadata; - std::shared_ptr _messageStore; + std::shared_ptr _messageStore; std::string _resultTableName; std::string _userQueryId; std::string _rowsTable; 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 182653a296..6731ea775c 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/CzarRegistry.cc b/src/czar/CzarRegistry.cc index 368bf66f01..f5abfcaba6 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -188,4 +188,11 @@ 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(); +} + } // namespace lsst::qserv::czar diff --git a/src/czar/CzarRegistry.h b/src/czar/CzarRegistry.h index 11c02d57d6..27d20979cf 100644 --- a/src/czar/CzarRegistry.h +++ b/src/czar/CzarRegistry.h @@ -87,6 +87,7 @@ class CzarRegistry { return (wId == other.wId && wHost == other.wHost && wManagementHost == other.wManagementHost && wPort == other.wPort); } + std::string dump() const; }; using WorkerContactMap = std::unordered_map; diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 7361665c61..d722d37042 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -77,6 +77,7 @@ json HttpCzarWorkerModule::_queryJobError() { checkApiVersion(__func__, 34); LOGS(_log, LOG_LVL_INFO, __func__ << "&&&uj queryJobError json=" << body().objJson); //&&& //&&&uj NEED CODE for this + auto ret = _handleJobError(__func__); return json::object(); } @@ -91,11 +92,59 @@ json HttpCzarWorkerModule::_queryJobReady() { //&&&return json::object(); } +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. + auto const& js = body().objJson; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR js=" << js); + string const targetWorkerId = body().required("workerid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR targetWorkerId=" << targetWorkerId); + string const czarName = body().required("czar"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarName=" << czarName); + qmeta::CzarId const czarId = body().required("czarid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarId=" << czarId); + QueryId const queryId = body().required("queryid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR queryId=" << queryId); + UberJobId const uberJobId = body().required("uberjobid"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR uberJobId=" << uberJobId); + int const errorCode = body().required("errorCode"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR errorCode=" << errorCode); + string const errorMsg = body().required("errorMsg"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR errorMsg=" << errorMsg); + + // Find UberJob + qdisp::Executive::Ptr exec = qdisp::Executive::getExecutiveFromMap(queryId); + if (exec == nullptr) { + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobError No executive for qid=") + + to_string(queryId)); + } + 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)); + } + // &&&uj NEED CODE to verify incoming values to those in the UberJob + + 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", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; + json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { // See qdisp::UberJob::runUberJob() for json message construction. auto const& js = body().objJson; @@ -106,7 +155,7 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarName=" << czarName); qmeta::CzarId const czarId = body().required("czarid"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarId=" << czarId); - QueryId const queryId = body().required("queryid"); + QueryId const queryId = body().required("queryid"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR queryId=" << queryId); UberJobId const uberJobId = body().required("uberjobid"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR uberJobId=" << uberJobId); @@ -117,31 +166,28 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { uint64_t const fileSize = body().required("fileSize"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR fileSize=" << fileSize); - // Find UberJob qdisp::Executive::Ptr exec = qdisp::Executive::getExecutiveFromMap(queryId); if (exec == nullptr) { - throw invalid_argument(string("HttpCzarWorkerModule::_handleJobRead No executive for qid=") + to_string(queryId)); + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No executive for qid=") + + to_string(queryId)); } qdisp::UberJob::Ptr uj = exec->findUberJob(uberJobId); if (uj == nullptr) { - throw invalid_argument(string("HttpCzarWorkerModule::_handleJobRead No UberJob for qid=") + to_string(queryId) + " ujId=" + to_string(uberJobId)); + throw invalid_argument(string("HttpCzarWorkerModule::_handleJobReady No UberJob for qid=") + + to_string(queryId) + " ujId=" + to_string(uberJobId)); } // &&&uj NEED CODE to verify incoming values to those in the UberJob - - - - auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); // &&&uj move this to after parse. + auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); jsRet = importRes; - - } catch (std::invalid_argument const& iaExp) { - LOGS(_log, LOG_LVL_ERROR, "HttpCzarWorkerModule::_handleJobReady received " << iaExp.what() << " js=" << body().objJson); - jsRet = {{"success", 0}, {"errortype", "parse"}, {"note", iaExp.what()}}; + } 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 8efac50c6a..9e6fc831a0 100644 --- a/src/czar/HttpCzarWorkerModule.h +++ b/src/czar/HttpCzarWorkerModule.h @@ -72,6 +72,8 @@ class HttpCzarWorkerModule : public czar::HttpModule { /// &&& doc nlohmann::json _handleJobReady(std::string const& func); + + nlohmann::json _handleJobError(std::string const& func); }; } // namespace lsst::qserv::czar 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/global/intTypes.h b/src/global/intTypes.h index 8074009d05..1693940539 100644 --- a/src/global/intTypes.h +++ b/src/global/intTypes.h @@ -38,7 +38,7 @@ 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. /// Class to provide a consistent format for QueryIds in the log file class QueryIdHelper { diff --git a/src/qdisp/CMakeLists.txt b/src/qdisp/CMakeLists.txt index 8d6456e196..e0aa446672 100644 --- a/src/qdisp/CMakeLists.txt +++ b/src/qdisp/CMakeLists.txt @@ -8,8 +8,6 @@ target_sources(qdisp PRIVATE JobBase.cc JobDescription.cc JobQuery.cc - JobStatus.cc - MessageStore.cc QdispPool.cc QueryRequest.cc UberJob.cc diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index b433b2b9d0..a752b57508 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -58,21 +58,24 @@ // 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/UberJob.h" #include "qdisp/XrdSsiMocks.h" #include "query/QueryContext.h" #include "qproc/QuerySession.h" #include "qmeta/Exceptions.h" +#include "qmeta/MessageStore.h" #include "qmeta/QStatus.h" #include "query/SelectStmt.h" +#include "rproc/InfileMerger.h" #include "util/AsyncTimer.h" #include "util/Bug.h" #include "util/EventThread.h" @@ -97,14 +100,13 @@ string getErrorText(XrdSsiErrInfo& e) { namespace lsst::qserv::qdisp { -mutex Executive::_executiveMapMtx; ///< protects _executiveMap -map Executive::_executiveMap; ///< Map of executives for queries in progress. - +mutex Executive::_executiveMapMtx; ///< protects _executiveMap +map Executive::_executiveMap; ///< Map of executives for queries in progress. //////////////////////////////////////////////////////////////////////// // class Executive implementation //////////////////////////////////////////////////////////////////////// -Executive::Executive(ExecutiveConfig const& c, shared_ptr const& ms, +Executive::Executive(ExecutiveConfig const& c, shared_ptr const& ms, SharedResources::Ptr const& sharedResources, shared_ptr const& qStatus, shared_ptr const& querySession) : _config(c), @@ -119,6 +121,7 @@ Executive::Executive(ExecutiveConfig const& c, shared_ptr const& m } Executive::~Executive() { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::~Executive() " << getIdStr()); qdisp::CzarStats::get()->deleteQuery(); qdisp::CzarStats::get()->deleteJobs(_incompleteJobs.size()); // Real XrdSsiService objects are unowned, but mocks are allocated in _setup. @@ -129,7 +132,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& qMeta, shared_ptr const& querySession, @@ -151,13 +154,14 @@ Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_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(); @@ -172,7 +176,8 @@ 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)); + 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); @@ -222,7 +227,7 @@ 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); @@ -300,6 +305,7 @@ void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect end"); } +/// &&&uj &&&&&&&&&&&&&&&&&&&&&& NEED CODE put this as command in qdisppool. void Executive::runUberJob(std::shared_ptr const& uberJob) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); bool started = uberJob->runUberJob(); @@ -360,12 +366,16 @@ bool Executive::startQuery(shared_ptr const& jobQuery) { // &&& return true; } -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) { // &&& @@ -376,6 +386,31 @@ void Executive::addUberJobs(std::vector> const& uJobsTo } } +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(); + /// &&& NEED CODE put on qdisppool &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&& + uqs->buildAndSendUberJobs(); +} + bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& lock_guard lock(_cancelled.getMutex()); @@ -419,9 +454,11 @@ 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); + LOGS(_log, LOG_LVL_WARN, "&&& entry state:" << (void*)entry.second.get() << " " << esI); + return (esI.state == qmeta::JobStatus::RESPONSE_DONE) || + (esI.state == qmeta::JobStatus::COMPLETE); } }; @@ -479,13 +516,18 @@ void Executive::markCompleted(JobId 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(), + if (jState != qmeta::JobStatus::CANCEL && jState != qmeta::JobStatus::RESPONSE_ERROR && + jState != qmeta::JobStatus::RESULT_ERROR && jState != qmeta::JobStatus::MERGE_ERROR) { + job->getStatus()->updateInfo(id, qmeta::JobStatus::RESULT_ERROR, "EXECFAIL", err.getCode(), err.getMsg()); } + */ + // Don't overwrite existing error states. + job->getStatus()->updateInfoNoErrorOverwrite(id, qmeta::JobStatus::RESULT_ERROR, "EXECFAIL", + err.getCode(), err.getMsg()); } { lock_guard lock(_errorsMutex); @@ -504,13 +546,14 @@ void Executive::markCompleted(JobId jobId, bool success) { } void Executive::squash() { + LOGS(_log, LOG_LVL_WARN, "&&&uj Executive::squash() qid=" << getId() << " str=" << getIdStr()); 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); @@ -539,7 +582,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); } } @@ -797,7 +841,7 @@ 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; } @@ -810,4 +854,194 @@ void Executive::_printState(ostream& os) { } } +#if 0 //&&& +void Executive::buildAndSendUberJobs(int const maxChunksPerUber) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs a"); + vector uberJobs; + + auto czarPtr = czar::Czar::getCzar(); + auto czChunkMap = czarPtr->getCzarChunkMap(); + auto czRegistry = czarPtr->getCzarRegistry(); + + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs b"); + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj + + // Make a map of all jobs in the executive. + // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can + // be found for all databases in the query + /* &&& instead of destroying the only copy, going to make a copy that only contains + * unassigned jobs. + qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); + if (_chunkToJobMapInvalid.exchange(true)) { + throw util::Bug(ERR_LOC, "getChunkJobMapInvalidate called when map already invalid"); + } + return _chunkToJobMap; + + */ + ChunkIdJobMapType unassignedChunksInQuery; + { + lock_guard lckChunkJobMap(_chunkToJobMapMtx); + for(auto const& [chunkId, jobPtr] : _chunkToJobMap) { + if (!jobPtr->isInUberJob()) { + unassignedChunksInQuery[chunkId] = jobPtr; + } + } + } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs c"); + + // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use + // the same databases) Use this to check for conflicts + + /* &&& + // assign jobs to uberJobs + int maxChunksPerUber = 3; // &&&uj maybe put in config??? or set on command line?? + // &&&uj Different queries may benefit from different values + // &&&uj Such as LIMIT=1 may work best with this at 1, where + // &&&uj 100 would be better for others. + */ + // keep cycling through workers until no more chunks to place. + + // TODO:UJ &&&uj Once everything is an UberJob, it can start with 1 or 0. + //int _uberJobId = qdisp::UberJob::getFirstIdNumber(); + + // &&&uj + // - 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. + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d"); + for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d1"); + auto iter = chunkMapPtr->find(chunkId); + if (iter == chunkMapPtr->end()) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d1a"); + missingChunks.push_back(chunkId); + break; + } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d2");; + czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; + auto targetWorker = chunkData->getPrimaryScanWorker().lock(); + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d3"); + if (targetWorker == nullptr) { //&&&uj if (targetWorker == nullptr || this worker already tried for this chunk) { + LOGS(_log, LOG_LVL_ERROR, "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, "Alternate worker found for chunk=" << chunkData->dump()); + } + } + if (!found) { + // &&&uj If enough workers are down, there will be a chunk that cannot be found. + // the correct course of action is probably to check the Registry, and + // after so many attempts, cancel the user query with a + // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount + // in the Job or JobDescription could be used for this. + LOGS(_log, LOG_LVL_ERROR, + "No primary or alternate worker found for chunk=" << chunkData->dump()); + throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + + " Crashing the program here for this reason is not " + "appropriate. &&& NEEDS CODE"); + } + } + // Add this job to the appropriate UberJob, making the UberJob if needed. + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d4"); + string workerId = targetWorker->getWorkerId(); + auto& ujVect = workerJobMap[workerId]; + if (ujVect.empty() || ujVect.back()->getJobCount() > maxChunksPerUber) { + LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d4a"); + auto ujId = _uberJobId++; // keep ujId consistent + string uberResultName = _ttn->make(ujId); + auto respHandler = make_shared(_infileMerger, uberResultName); + auto thisExec = shared_from_this(); + auto uJob = qdisp::UberJob::create(thisExec, respHandler, getId(), ujId, + _qMetaCzarId, targetWorker); + ujVect.push_back(uJob); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c6"); + ujVect.back()->addJob(jqPtr); + } + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c7"); + if (!missingChunks.empty()) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c8"); + string errStr = string(__func__) + " a worker could not be found for these chunks "; + for (auto const& chk : missingChunks) { + errStr += to_string(chk) + ","; + } + LOGS(_log, LOG_LVL_ERROR, errStr); + throw util::Bug( + ERR_LOC, + errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); + + //&&&uj + // Add worker contact info to UberJobs. + auto const wContactMap = czRegistry->getWorkerContactMap(); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f"); + for (auto const& [wIdKey, ujVect] : workerJobMap) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f1"); + auto iter = wContactMap->find(wIdKey); + if (iter == wContactMap->end()) { + // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. + throw util::Bug(ERR_LOC, string(" &&&uj NEED CODE, no contact information for ") + wIdKey); + } + auto const& wContactInfo = iter->second; + for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f2"); + ujPtr->setWorkerContactInfo(wContactInfo); + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f3"); + _executive->addUberJobs(ujVect); + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f4"); + for (auto const& ujPtr : ujVect) { + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f5"); + //&&&uj IMPORTANT + //&&&uj This just sends the test message to the worker, but the + // worker only parses it and sends a message back. The + // worker does not create and run tasks at this point. + // The call to runUberJob here should be replaced by a call + // to startUberJob that puts the call to runUberJob into + // the a priority queue command. + _executive->runUberJob(ujPtr); + } + } + + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj + + LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); + _executive->waitForAllJobsToStart(); + + // we only care about per-chunk info for ASYNC queries + if (_async) { + std::lock_guard lock(chunksMtx); + _qMetaAddChunks(chunks); + } +} +#endif //&&& + } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 14f85d161c..c413ccf7ee 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -40,10 +40,11 @@ #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" @@ -58,9 +59,14 @@ class XrdSsiService; namespace lsst::qserv { +namespace ccontrol { +class UserQuerySelect; +} + namespace qmeta { +class MessageStore; class QStatus; -} +} // namespace qmeta namespace qproc { class QuerySession; @@ -68,10 +74,13 @@ class QuerySession; namespace qdisp { class JobQuery; -class MessageStore; class UberJob; //&&&uj } // namespace qdisp +namespace rproc { +class InfileMerger; +} + namespace util { class AsyncTimer; } @@ -101,7 +110,7 @@ class Executive : public std::enable_shared_from_this { /// 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& qMeta, std::shared_ptr const& querySession, @@ -112,6 +121,13 @@ class Executive : public std::enable_shared_from_this { /// &&& doc static Ptr getExecutiveFromMap(QueryId qId); + /// &&&uj doc + void setUserQuerySelect(std::shared_ptr const& uqs) { _userQuerySelect = uqs; } + //&&&void buildAndSendUberJobs(int const maxChunksPerUber); + + /// &&&uj doc Return a map that only contains Jobs not assigned to an UberJob. + ChunkIdJobMapType unassignedChunksInQuery(); + /// &&& doc void removeFromMap(); @@ -149,7 +165,15 @@ class Executive : public std::enable_shared_from_this { 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); + //&&&void setTmpTableNameGenerator(std::shared_ptr const& ttn) { _ttn = ttn; } + + //&&&void setInfileMerger(std::shared_ptr infileMerger) { _infileMerger = + //infileMerger; } + QueryId getId() const { return _id; } std::string const& getIdStr() const { return _idStr; } @@ -193,12 +217,21 @@ class Executive : public std::enable_shared_from_this { /// Add UbjerJobs to this user query. &&& void addUberJobs(std::vector> const& jobsToAdd); - ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& + /// &&&uj doc + void assignJobsToUberJobs(); + + ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& delete bool startUberJob(std::shared_ptr const& uJob); /// &&& std::shared_ptr getSharedPtrForRawJobPtr(JobQuery* jqRaw); /// &&& + int getTotalJobs() { return _totalJobs; } + + void setFlagFailedUberJob(bool val) { _failedUberJob = val; } + + std::string dumpUberJobCounts() const; + 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& qStatus, std::shared_ptr const& querySession); @@ -228,7 +261,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. @@ -255,11 +288,25 @@ class Executive : public std::enable_shared_from_this { mutable std::mutex _errorsMutex; std::condition_variable _allJobsComplete; - mutable std::recursive_mutex _jobMapMtx; + mutable std::recursive_mutex _jobMapMtx; // &&& see what it takes to make this a normal mutex - 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"}; + + /* &&& + qmeta::CzarId _qMetaCzarId; ///< Czar ID in QMeta database + + /// temporary table name generator, which uses a hash and jobId or uberJobId + /// to generate names for tables. This cannot be set until after `_id` is + /// set. + std::shared_ptr _ttn; + + /// Pointer to the result merging class. + std::shared_ptr _infileMerger; + + /// UberJobIds for need to be unique within each UserQuery. + std::atomic _uberJobId = qdisp::UberJob::getFirstIdNumber(); + */ std::shared_ptr _qMeta; /// Last time Executive updated QMeta, defaults to epoch for clock. @@ -273,14 +320,15 @@ class Executive : public std::enable_shared_from_this { // Add a job to the _chunkToJobMap //&&&uj void _addToChunkJobMap(std::shared_ptr const& job); //&&&uj /// _chunkToJobMap is created once and then destroyed after use. - 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::atomic _chunkToJobMapInvalid{ + false}; ///< true indicates the map is no longer valid. //&&&uj delete + std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap //&&&uj + ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId //&&&uj /// Map of all UberJobs. Failed UberJobs remain in the map as new ones are created /// to handle failed UberJobs. std::map> _uberJobsMap; - std::mutex _uberJobsMapMtx; ///< protects _uberJobs. //&&&uj + mutable std::mutex _uberJobsMapMtx; ///< protects _uberJobs. //&&&uj /// True if enough rows were read to satisfy a LIMIT query with /// no ORDER BY or GROUP BY clauses. @@ -297,12 +345,22 @@ 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. + std::atomic _queryIdSet{false}; ///< Set to true when _id is set. + + ///&&&uj + std::weak_ptr _userQuerySelect; + + /// If true, there are probably jobs that need to be reassigned to new + /// UberJobs. + /// &&&uj NEED CODE - at some point this needs to be checked so the + /// executive can make new uberjobs. + std::atomic _failedUberJob{false}; - static std::mutex _executiveMapMtx; ///< protects _executiveMap - static std::map _executiveMap; ///< Map of executives for queries in progress. + static std::mutex _executiveMapMtx; ///< protects _executiveMap + static std::map _executiveMap; ///< Map of executives for queries in progress. }; +/// &&&uj MarkCompleteFunc is not needed with uberjobs. class MarkCompleteFunc { public: typedef std::shared_ptr Ptr; diff --git a/src/qdisp/JobBase.h b/src/qdisp/JobBase.h index cce8d31ead..e5df5fc2ab 100644 --- a/src/qdisp/JobBase.h +++ b/src/qdisp/JobBase.h @@ -28,11 +28,14 @@ // Qserv headers #include "global/intTypes.h" +namespace lsst::qserv::qmeta { +class JobStatus; +} + // This header declarations -namespace lsst { namespace qserv { namespace qdisp { +namespace lsst::qserv::qdisp { class Executive; -class JobStatus; class QdispPool; class ResponseHandler; class QueryRequest; @@ -57,7 +60,7 @@ class JobBase : public std::enable_shared_from_this { 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 std::shared_ptr getStatus() = 0; virtual bool getScanInteractive() const = 0; virtual bool isQueryCancelled() = 0; virtual void callMarkCompleteFunc(bool success) = 0; @@ -70,6 +73,6 @@ class JobBase : public std::enable_shared_from_this { friend std::ostream& operator<<(std::ostream& os, JobBase const& jb); }; -}}} // namespace lsst::qserv::qdisp +} // namespace lsst::qserv::qdisp #endif // LSST_QSERV_QDISP_JOBBASE_H diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 6f93bdf1dd..aa4deaa0f5 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -45,8 +45,8 @@ using namespace std; namespace lsst::qserv::qdisp { 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), @@ -108,7 +108,7 @@ bool JobQuery::runJob() { // &&& 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; @@ -173,6 +173,34 @@ bool JobQuery::isQueryCancelled() { return exec->getCancelled(); } +bool JobQuery::_setUberJobId(UberJobId ujId) { + QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); + if (_uberJobId >= 0 && ujId != _uberJobId) { + LOGS(_log, LOG_LVL_ERROR, + __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; + _jobDescription->incrAttemptCountScrubResultsJson(); + return true; +} + string const& JobQuery::getPayload() const { return _jobDescription->payload(); } void JobQuery::callMarkCompleteFunc(bool success) { _markCompleteFunc->operator()(success); } diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 32feee158f..0a62eafd07 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -43,16 +43,17 @@ namespace lsst::qserv::qdisp { class QdispPool; class QueryRequest; -/** This class is used to describe, monitor, and control a single query to a worker. - * - */ +/// This class is used to describe, monitor, and control a single query to a worker. +/// &&&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 = Ptr(new JobQuery(executive, jobDescription, jobStatus, markCompleteFunc, qid)); jq->_setup(); @@ -72,7 +73,7 @@ class JobQuery : public JobBase { bool getScanInteractive() const override { return _jobDescription->getScanInteractive(); } JobDescription::Ptr getDescription() { return _jobDescription; } - JobStatus::Ptr getStatus() override { return _jobStatus; } + qmeta::JobStatus::Ptr getStatus() override { return _jobStatus; } void setQueryRequest(std::shared_ptr const& qr) { std::lock_guard lock(_rmutex); @@ -97,14 +98,28 @@ class JobQuery : public JobBase { /// 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); - /// Set to true if this job is part of an UberJob - void setInUberJob(bool inUberJob) { _inUberJob = inUberJob; }; + /// If the UberJob is unassigned, change the _uberJobId to ujId. + bool setUberJobId(UberJobId ujId) { + std::lock_guard lock(_rmutex); + return _setUberJobId(ujId); + } - /// @return true if this job is part of an UberJob. - bool inUberJob() const { return _inUberJob; } + UberJobId getUberJobId() const { + std::lock_guard lock(_rmutex); + return _getUberJobId(); + } + + bool isInUberJob() const { + std::lock_guard lock(_rmutex); + return _isInUberJob(); + } + + /// If ujId is the current owner, clear ownership. + /// @return true if job is unassigned. + bool unassignFromUberJob(UberJobId ujId); protected: void _setup() { @@ -112,6 +127,7 @@ class JobQuery : public JobBase { _jobDescription->respHandler()->setJobQuery(jbPtr); } + /// NOTE: _rmutex must be held before calling this int _getRunAttemptsCount() const { std::lock_guard lock(_rmutex); return _jobDescription->getAttemptCount(); @@ -119,6 +135,17 @@ class JobQuery : public JobBase { 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. + /// @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. @@ -126,14 +153,16 @@ class JobQuery : public JobBase { 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 + // &&&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; @@ -144,9 +173,11 @@ class JobQuery : public JobBase { std::shared_ptr _qdispPool; - /// True if this job is part of an UberJob. - std::atomic _inUberJob{ - false}; ///< TODO:UJ There are probably several places this should be checked + /// 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. + UberJobId _uberJobId = -1; }; } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/QueryRequest.cc b/src/qdisp/QueryRequest.cc index 1ab2559a91..7da1046846 100644 --- a/src/qdisp/QueryRequest.cc +++ b/src/qdisp/QueryRequest.cc @@ -46,7 +46,7 @@ #include "qdisp/UberJob.h" #include "global/LogContext.h" #include "proto/worker.pb.h" -#include "qdisp/JobStatus.h" +#include "qmeta/JobStatus.h" #include "qdisp/ResponseHandler.h" #include "util/Bug.h" #include "util/common.h" @@ -126,7 +126,7 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co os << _jobIdStr << __func__ << " request failed " << getSsiErr(eInfo, nullptr) << " " << GetEndPoint(); jq->getRespHandler()->errorFlush(os.str(), -1); - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_ERROR, "SSI"); + jq->getStatus()->updateInfo(_jobIdStr, qmeta::JobStatus::RESPONSE_ERROR, "SSI"); _errorFinish(); return true; } @@ -138,18 +138,18 @@ bool QueryRequest::ProcessResponse(XrdSsiErrInfo const& eInfo, XrdSsiRespInfo co break; case XrdSsiRespInfo::isData: if (string(rInfo.buff, rInfo.blen) == "MockResponse") { - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::COMPLETE, "MOCK"); + jq->getStatus()->updateInfo(_jobIdStr, qmeta::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"); + jq->getStatus()->updateInfo(_jobIdStr, qmeta::JobStatus::RESPONSE_READY, "SSI"); return _importResultFile(jq); } responseTypeName = "isData"; break; case XrdSsiRespInfo::isError: - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_ERROR, "SSI", rInfo.eNum, + jq->getStatus()->updateInfo(_jobIdStr, qmeta::JobStatus::RESPONSE_ERROR, "SSI", rInfo.eNum, string(rInfo.eMsg)); return _importError(string(rInfo.eMsg), rInfo.eNum); case XrdSsiRespInfo::isFile: @@ -214,7 +214,7 @@ bool QueryRequest::_importResultFile(JobBase::Ptr const& job) { // 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"); + jq->getStatus()->updateInfo(_jobIdStr, qmeta::JobStatus::COMPLETE, "COMPLETE"); _finish(); // If the query meets the limit row complete complete criteria, it will start @@ -249,8 +249,8 @@ void QueryRequest::ProcessResponseData(XrdSsiErrInfo const& eInfo, char* buff, i void QueryRequest::_flushError(JobBase::Ptr const& jq) { ResponseHandler::Error err = jq->getRespHandler()->getError(); - jq->getStatus()->updateInfo(_jobIdStr, JobStatus::MERGE_ERROR, "MERGE", err.getCode(), err.getMsg(), - MSG_ERROR); + jq->getStatus()->updateInfo(_jobIdStr, qmeta::JobStatus::MERGE_ERROR, "MERGE", err.getCode(), + err.getMsg(), MSG_ERROR); _errorFinish(true); } @@ -268,7 +268,7 @@ bool QueryRequest::cancel() { // Only call the following if the job is NOT already done. if (_finishStatus == ACTIVE) { auto jq = _job; - if (jq != nullptr) jq->getStatus()->updateInfo(_jobIdStr, JobStatus::CANCEL, "CANCEL"); + if (jq != nullptr) jq->getStatus()->updateInfo(_jobIdStr, qmeta::JobStatus::CANCEL, "CANCEL"); } } return _errorFinish(true); // return true if errorFinish cancelled diff --git a/src/qdisp/QueryRequest.h b/src/qdisp/QueryRequest.h index b6bbd2c2c7..74826d16cd 100644 --- a/src/qdisp/QueryRequest.h +++ b/src/qdisp/QueryRequest.h @@ -159,7 +159,7 @@ class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this _keepAlive; ///< Used to keep this object alive during race condition. QueryId _qid = 0; // for logging - JobId _jobid = -1; // for logging + JobId _jobid = -1; // for logging std::string _jobIdStr{QueryIdHelper::makeIdStr(0, 0, true)}; ///< for debugging only. std::atomic _finishedCalled{false}; diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index afffc21e93..d549947025 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -71,7 +71,8 @@ class ResponseHandler { /// @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 expectedRows, + uint64_t& resultRows) = 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 3536ae4860..3d3b81c1a0 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -71,11 +71,9 @@ UberJob::UberJob(Executive::Ptr const& executive, std::shared_ptr jobsLock(_jobsMtx); + LOGS(_log, LOG_LVL_WARN, + getIdStr() << "&&&uj count qid=" << getQueryId() << " ujId=" << getJobId() + << " jobs.sz=" << _jobs.size()); for (auto const& jqPtr : _jobs) { - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() a1"); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() a1"); jqPtr->getDescription()->incrAttemptCountScrubResultsJson(); } - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() b"); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() b"); // Send the uberjob to the worker auto const method = http::Method::POST; string const url = "http://" + _wContactInfo->wHost + ":" + to_string(_wContactInfo->wPort) + "/queryjob"; - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() c " << url); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() c " << url); vector const headers = {"Content-Type: application/json"}; auto const& czarConfig = cconfig::CzarConfig::instance(); - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() c"); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() c"); // See xrdsvc::httpWorkerCzarModule::_handleQueryJob for json message parsing. json request = {{"version", http::MetaModule::version}, {"instance_id", czarConfig->replicationInstanceId()}, @@ -132,26 +135,38 @@ bool UberJob::runUberJob() { {"czarid", _czarId}, {"jobs", json::array()}}}}; - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() d " << request); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() d " << request); auto& jsUberJob = request["uberjob"]; - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() e " << jsUberJob); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() e " << jsUberJob); auto& jsJobs = jsUberJob["jobs"]; - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() f " << jsJobs); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() f " << jsJobs); for (auto const& jbPtr : _jobs) { - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() f1"); - json jsJob = {{"jobdesc", *(jbPtr->getDescription()->getJsForWorker())}}; + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1"); + auto const description = jbPtr->getDescription(); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1a"); + if (description == nullptr) { + throw util::Bug(ERR_LOC, getIdStr() + " &&&uj description=null for job=" + jbPtr->getIdStr()); + } + auto const jsForWorker = jbPtr->getDescription()->getJsForWorker(); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1b"); + if (jsForWorker == nullptr) { + throw util::Bug(ERR_LOC, getIdStr() + " &&&uj jsForWorker=null for job=" + jbPtr->getIdStr()); + } + //&&& json jsJob = {{"jobdesc", *(jbPtr->getDescription()->getJsForWorker())}}; + json jsJob = {{"jobdesc", *jsForWorker}}; jsJobs.push_back(jsJob); jbPtr->getDescription()->resetJsForWorker(); // no longer needed. } - jobsLock.unlock(); + jobsLock.unlock(); // unlock so other _jobsMtx threads can advance while this waits for transmit - LOGS(_log, LOG_LVL_WARN, "&&& UberJob::runUberJob() g"); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() g"); LOGS(_log, LOG_LVL_WARN, __func__ << " &&&REQ " << request); 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); bool transmitSuccess = false; + string exceptionWhat; try { json const response = client.readAsJson(); LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() response=" << response); @@ -159,76 +174,24 @@ bool UberJob::runUberJob() { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() success"); transmitSuccess = true; } else { - LOGS(_log, LOG_LVL_WARN, "&&&uj NEED CODE UberJob::runUberJob() success=0"); + LOGS(_log, LOG_LVL_WARN, _idStr << " UberJob::" << __func__ << " response success=0"); } } catch (exception const& ex) { LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); + exceptionWhat = ex.what(); } if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, "&&&uj NEED CODE UberJob::runUberJob() need to try to send jobs elsewhere"); - } else { - LOGS(_log, LOG_LVL_WARN, - "&&&uj NEED CODE UberJob::runUberJob() need to register all jobs as transmitted to worker"); - } + LOGS(_log, LOG_LVL_ERROR, "&&&uj UberJob::runUberJob() transmit failure, try to send jobs elsewhere"); + LOGS(_log, LOG_LVL_ERROR, "UberJob::" << __func__ << " transmit failure, try to send jobs elsewhere"); + _unassignJobs(); // locks _jobsMtx + setStatusIfOk(qmeta::JobStatus::RESPONSE_ERROR, + string("UberJob::") + __func__ + " not transmitSuccess " + exceptionWhat); -#if 0 // &&&uj Everything in this block needs to happen in some manner. Where is the question - // For simplicity in the first pass, just make a TaskMsg for each Job and append it to the - // UberJobMsg. - // This is terribly inefficient and should be replaced by using a template and list of chunks that - // the worker fills in, much like subchunks are done now. - { - google::protobuf::Arena arena; - proto::UberJobMsg* ujMsg = google::protobuf::Arena::CreateMessage(&arena); - ujMsg->set_queryid(getQueryId()); - ujMsg->set_czarid(_czarId); - ujMsg->set_uberjobid(_uberJobId); - ujMsg->set_magicnumber(UberJob::getMagicNumber()); - LOGS(_log, LOG_LVL_INFO, "&&& runUberJob sz=" << _jobs.size()); - for (auto&& job : _jobs) { - proto::TaskMsg* tMsg = ujMsg->add_taskmsgs(); - job->getDescription()->fillTaskMsg(tMsg); - } - ujMsg->SerializeToString(&_payload); - } - - auto executive = _executive.lock(); - if (executive == nullptr) { - LOGS(_log, LOG_LVL_ERROR, "runUberJob failed executive==nullptr"); - return false; - } - bool cancelled = executive->getCancelled(); - bool handlerReset = _respHandler->reset(); - bool started = _started.exchange(true); - if (!cancelled && handlerReset && !started) { - auto criticalErr = [this, &executive](std::string const& msg) { - LOGS(_log, LOG_LVL_ERROR, msg << " " << *this << " Canceling user query!"); - executive->squash(); // This should kill all jobs in this user query. - }; - - LOGS(_log, LOG_LVL_DEBUG, "runUberJob verifying payloads"); - if (!verifyPayload()) { - criticalErr("bad payload"); - return false; - } - - // At this point we are all set to actually run the queries. 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, "runUberJob calls StartQuery()"); - std::shared_ptr uJob(dynamic_pointer_cast(shared_from_this())); - _inSsi = true; - if (executive->startUberJob(uJob)) { - //&&&_jobStatus->updateInfo(_idStr, JobStatus::REQUEST); - _jobStatus->updateInfo(_idStr, JobStatus::REQUEST, "EXEC"); - return true; - } - _inSsi = false; + } else { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() register all jobs as transmitted to worker"); + setStatusIfOk(qmeta::JobStatus::REQUEST, + string("UberJob::") + __func__ + " transmitSuccess"); // locks _jobsMtx } - LOGS(_log, LOG_LVL_WARN, - "runUberJob failed. cancelled=" << cancelled << " reset=" << handlerReset << " started=" << started); -#endif // &&& return false; } @@ -240,6 +203,27 @@ void UberJob::prepScrubResults() { "&&&uj If needed, should call prepScrubResults for all JobQueries in the UberJob "); } +void UberJob::_unassignJobs() { + lock_guard lck(_jobsMtx); + for (auto&& job : _jobs) { + string idStr = job->getIdStr(); + if (!job->unassignFromUberJob(getJobId())) { + auto exec = _executive.lock(); + if (exec != nullptr) { + LOGS(_log, LOG_LVL_ERROR, " max attempts for " << idStr << " reached, cancelling"); + exec->squash(); + return; + } + } + } + _jobs.clear(); + auto const exec = _executive.lock(); + if (exec != nullptr) { + bool const setFlag = true; + exec->setFlagFailedUberJob(setFlag); + } +} + bool UberJob::isQueryCancelled() { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); auto exec = _executive.lock(); @@ -259,54 +243,77 @@ bool UberJob::verifyPayload() const { return true; } -void UberJob::callMarkCompleteFunc(bool success) { // &&&uj make private - LOGS(_log, LOG_LVL_DEBUG, "UberJob::callMarkCompleteFunc success=" << success); - if (!success) { - // &&&uj this function should probably only be called for successful completion. - throw util::Bug(ERR_LOC, "&&&NEED_CODE may need code to properly handle failed uberjob"); +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, + getIdStr() << "UberJob::" << __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, + 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. + // &&&uj JobStatus should have a separate entry for success/failure/incomplete/retry. + string source = string("UberJob_") + (success ? "SUCCESS" : "FAILED"); + _jobStatus->updateInfo(getIdStr(), qmeta::JobStatus::COMPLETE, source); for (auto&& job : _jobs) { string idStr = job->getIdStr(); - job->getStatus()->updateInfo(idStr, JobStatus::COMPLETE, "COMPLETE"); - job->callMarkCompleteFunc(success); + 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(); - - //&&&uj NEED CODE _resultStatus = MERGESUCCESS; - //&&&uj NEED CODE _status = COMPLETE; } - /// 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(JobBase::Ptr const& job) { -nlohmann::json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { +json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile a"); - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); - - //&&&uj NEED CODE update status for each job in this uberjob - // jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); + LOGS(_log, LOG_LVL_WARN, + "&&&uj UberJob::importResultFile fileUrl=" << fileUrl << " rowCount=" << rowCount + << " fileSize=" << fileSize); // It's possible jq and _jobQuery differ, so need to use jq. if (isQueryCancelled()) { LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile import job was cancelled."); - return _errorFinish(true); + return _importResultError(true, "cancelled", "Query cancelled"); } LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile b"); - /* &&& - auto jq = std::dynamic_pointer_cast(job); - if (jq == nullptr) { - throw util::Bug(ERR_LOC, string(__func__) + " unexpected pointer type for job"); - } - */ + auto exec = _executive.lock(); if (exec == nullptr || exec->getCancelled()) { LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile no executive or cancelled"); - return _errorFinish(true); + return _importResultError(true, "cancelled", "Query cancelled - no executive"); } LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile c"); @@ -314,200 +321,185 @@ nlohmann::json UberJob::importResultFile(string const& fileUrl, uint64_t rowCoun 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 _errorFinish(false); + return _importResultError(false, "rowLimited", "Enough rows already"); } LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile d"); - //&&& int messageSize = 0; - //&&& const char* message = GetMetadata(messageSize); LOGS(_log, LOG_LVL_DEBUG, __func__ << " _jobIdStr=" << getIdStr() << ", fileSize=" << fileSize); + bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); + if (!statusSet) { + LOGS(_log, LOG_LVL_WARN, getIdStr() << " &&&uj 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); - /// &&&&&&&&&&&&&&&&&&&&&&uj This NEEDS CODE Command class item instead of lambda and queue that to qdisppool &&&&&&&&&&&&&&&&& - /// &&&&&&&&&uj Also, HttpCzarWorkerModule::_handleJobReady isn't getting message from the worker UberJobData::fileReadyResponse &&&&&&&&& + // &&&uj lambda may not be the best way to do this. + // &&&uj check synchronization - may need a mutex for merging. auto fileCollectFunc = [ujThis, fileUrl, rowCount](util::CmdData*) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc a"); - /* &&& - // &&&uj this version of flush is going to have issues. - // &&&uj the reading of the file needs to happen elsewhere. - uint32_t resultRows = 0; - if (!jq->getDescription()->respHandler()->flush(responseSummary, resultRows)) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " not flushOk"); - _flushError(jq); - return false; - } - //&&&_totalRows += resultRows; - * - */ auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + 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); + auto [flushSuccess, flushShouldCancel] = + ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc b"); if (!flushSuccess) { // This would probably indicate malformed file+rowCount or // writing the result table failed. - ujPtr->_errorFinish(flushShouldCancel); + 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. - //&&&jq->getStatus()->updateInfo(_jobIdStr, JobStatus::COMPLETE, "COMPLETE"); LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc c"); - ujPtr->_finish(resultRows); //&&&uj flush and finish need to happen elsewhere, put it in qdisppool. - + ujPtr->_importResultFinish(resultRows); LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc end"); }; - //&&&fileCollectFunc(); - - //&&&auto cmd = std::make_shared(fileCollectFunc); auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); exec->queueFileCollect(cmd); - - /* &&&uj no need for this - proto::ResponseSummary responseSummary; - if (!(responseSummary.ParseFromArray(message, messageSize) && responseSummary.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); - } - */ - - // 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; +} -#if 0 //&&& - // It's possible jq and _jobQuery differ, so need to use jq. - if (job->isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, "QueryRequest::_processData job was cancelled."); - _errorFinish(true); - return false; - } - auto jq = std::dynamic_pointer_cast(job); - if (jq == nullptr) { - throw util::Bug(ERR_LOC, string(__func__) + " unexpected pointer type for job"); - } - 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; - } - - int messageSize = 0; - const char* message = GetMetadata(messageSize); +json UberJob::workerError(int errorCode, string const& errorMsg) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError a"); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError code=" << errorCode << " msg=" << errorMsg); + LOGS(_log, LOG_LVL_INFO, "UberJob::workerError code=" << errorCode << " msg=" << errorMsg); - LOGS(_log, LOG_LVL_DEBUG, __func__ << " _jobIdStr=" << _jobIdStr << ", messageSize=" << messageSize); + //&&&uj NEED CODE update status for each job in this uberjob + // jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); - proto::ResponseSummary responseSummary; - if (!(responseSummary.ParseFromArray(message, messageSize) && responseSummary.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); + bool const deleteData = true; + auto exec = _executive.lock(); + if (exec == nullptr || isQueryCancelled()) { + LOGS(_log, LOG_LVL_WARN, "UberJob::workerError no executive or cancelled"); + return _workerErrorFinish(deleteData, "cancelled"); } - uint32_t resultRows = 0; - if (!jq->getDescription()->respHandler()->flush(responseSummary, resultRows)) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " not flushOk"); - _flushError(jq); - return false; + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError c"); + + 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 _workerErrorFinish(!deleteData, "none", "limitRowComplete"); } - _totalRows += resultRows; - // 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(); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError d"); - // If the query meets the limit row complete complete criteria, it will start - // squashing superfluous results so the answer can be returned quickly. - executive->addResultRows(_totalRows); - executive->checkLimitRowComplete(); + //&&&uj get error message to the user? - return true; -#endif // &&& + JobBase::Ptr jBaseThis = shared_from_this(); + weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); + + string errType = to_string(errorCode) + ":" + errorMsg; + return _workerErrorFinish(deleteData, errType, ""); } -json UberJob::_errorFinish(bool shouldCancel) { - json jsRet = {{"success", 0}, {"errortype", "dataproblem"}, {"note", ""}}; - /// &&&uj NEED CODE - /// - 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. - /// &&&uj see QueryRequest for some details - /// If shouldCancel is false, it may be possible to recover, so all - /// jobs that were in this query should marked NEED_RETRY so they - /// will be retried. - /// If shouldCancel is true, this function should call markComplete - /// for all jobs in the uberjob, with all jobs failed. - /// - /// In both case, the worker should delete the file as - /// this czar will not ask for it, so return a "success:0" json - /// message to the worker. - if (shouldCancel) { - jsRet = {{"success", 0}, {"errortype", "cancelling"}, {"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. + + auto exec = _executive.lock(); + if (exec != nullptr) { + LOGS(_log, LOG_LVL_ERROR, + "UberJob::" << __func__ << " uberJobId=" << getJobId() << " shouldCancel=" << shouldCancel + << " errorType=" << errorType << " " << note); + if (shouldCancel) { + LOGS(_log, LOG_LVL_ERROR, + "UberJob::" << __func__ << " uberJobId=" << getJobId() << " 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, + "UberJob::" << __func__ << " uberJobId=" << getJobId() << " reassigning jobs"); + _unassignJobs(); + exec->assignJobsToUberJobs(); + } } else { - ; + LOGS(_log, LOG_LVL_INFO, + "UberJob::" << __func__ << " uberJobId=" << getJobId() << " already cancelled shouldCancel=" + << shouldCancel << " errorType=" << errorType << " " << note); } return jsRet; } -nlohmann::json UberJob::_finish(uint64_t resultRows) { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_finish a"); - /// &&&uj NEED CODE - /// &&&uj see QueryRequest for some details - /// If this is called, the file has been collected and the worker should delete it +nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish a"); + /// 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. + /// 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, "UberJob::" << __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, "UberJob::_finish executive is null qId=" << getQueryId() << " ujId=" << getJobId()); + LOGS(_log, LOG_LVL_DEBUG, "UberJob::" << __func__ << " executive is null " << getIdStr()); return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; } bool const success = true; - callMarkCompleteFunc(success); + callMarkCompleteFunc(success); // sets status to COMPLETE exec->addResultRows(resultRows); exec->checkLimitRowComplete(); - - json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_finish end"); + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish end"); return jsRet; } +nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, + std::string const& note) { + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_workerErrorFinish a"); + /// 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??? + /// &&& 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, + "UberJob::_workerErrorFinish executive is null qId=" << getQueryId() << " ujId=" << getJobId()); + return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; + } + + json jsRet = {{"success", 1}, {"deletedata", deleteData}, {"errortype", ""}, {"note", ""}}; + LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish end"); + 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(); diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index d276e10dd5..4a0ad126d7 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -27,8 +27,8 @@ #include "qmeta/types.h" #include "czar/CzarChunkMap.h" // Need nested class. &&&uj Make non-nested? #include "czar/CzarRegistry.h" // Need nested class. &&&uj Make non-nested? -//&&&#include "qdisp/Executive.h" #include "qdisp/JobBase.h" +#include "qmeta/JobStatus.h" // This header declarations namespace lsst::qserv::qdisp { @@ -37,15 +37,17 @@ class JobQuery; class QueryRequest; +/// &&& doc +/// 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. class UberJob : public JobBase { public: using Ptr = std::shared_ptr; - static uint32_t getMagicNumber() { return 93452; } - - 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); + 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; @@ -53,30 +55,37 @@ class UberJob : public JobBase { virtual ~UberJob(){}; - static int getFirstIdNumber() { return 9'000'000; } + static int getFirstIdNumber() { return 9'000'000; } // &&&uj this can probably be 0 now. bool addJob(std::shared_ptr const& job); bool runUberJob(); - QueryId getQueryId() const override { return _queryId; } // TODO:UJ relocate to JobBase - UberJobId getJobId() const override { return _uberJobId; } // &&&uj change name + QueryId getQueryId() const override { return _queryId; } // TODO:UJ relocate to JobBase + UberJobId getJobId() const override { return _uberJobId; } // &&&uj change name 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; } std::shared_ptr getRespHandler() override { return _respHandler; } - std::shared_ptr getStatus() override { return _jobStatus; } // TODO:UJ relocate to JobBase + 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 { - /* &&& - std::lock_guard lock(_qrMtx); - _queryRequestPtr = qr; - } - */ - ; // Do nothing as QueryRequest is only needed for xrootd. + ; // Do nothing as QueryRequest is only needed for xrootd. &&&uj + } + + /// 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); } bool verifyPayload() const; @@ -97,30 +106,47 @@ class UberJob : public JobBase { /// &&&uj doc nlohmann::json importResultFile(std::string const& fileUrl, uint64_t rowCount, uint64_t fileSize); + /// &&&uj doc + 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); + 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); + + /// &&&uj doc + void _unassignJobs(); + /// &&&uj doc - nlohmann::json _errorFinish(bool shouldCancel); + /// &&&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); /// &&&uj doc - nlohmann::json _finish(uint64_t resultRows); + nlohmann::json _importResultFinish(uint64_t resultRows); + /// &&& uj doc + nlohmann::json _workerErrorFinish(bool successful, std::string const& errorType = std::string(), + std::string const& note = std::string()); std::vector> _jobs; //&&&uj - std::mutex _jobsMtx; ///< Protects _jobs + mutable std::mutex _jobsMtx; ///< Protects _jobs, _jobStatus std::atomic _started{false}; bool _inSsi = false; - std::shared_ptr _jobStatus; // &&&uj uber jobstatus probably needs to different than jobstatus. + qmeta::JobStatus::Ptr _jobStatus{new qmeta::JobStatus()}; // &&&uj The JobStatus class should be changed + // to better represent UberJobs //&&& std::shared_ptr _queryRequestPtr; - std::mutex _qrMtx; + //&&&std::mutex _qrMtx; std::string _payload; ///< XrdSsi message to be sent to the _workerResource. //&&&uj remove when possible diff --git a/src/qdisp/testQDisp.cc b/src/qdisp/testQDisp.cc index 006e7efbc4..74483ab395 100644 --- a/src/qdisp/testQDisp.cc +++ b/src/qdisp/testQDisp.cc @@ -40,10 +40,10 @@ #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/MessageStore.h" #include "qproc/ChunkQuerySpec.h" #include "qproc/TaskMsgFactory.h" #include "util/threadSafe.h" @@ -160,7 +160,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; @@ -172,7 +172,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); @@ -212,7 +212,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); } @@ -257,7 +257,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"); @@ -265,7 +265,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"); } @@ -279,7 +279,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()); } @@ -292,7 +292,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); } @@ -305,7 +305,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! } @@ -322,7 +322,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..466bf34c01 100644 --- a/src/qhttp/Server.cc +++ b/src/qhttp/Server.cc @@ -277,7 +277,34 @@ 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); + LOGLS_DEBUG(_log, logger(this) << logger(request->_socket) << " &&&uj invoking handler for " + << pathHandler.path.regex << " " << request->path + << " match=" << pathMatch); + 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 a3c2ce25e1..9c4527ef9c 100644 --- a/src/qmeta/CMakeLists.txt +++ b/src/qmeta/CMakeLists.txt @@ -1,6 +1,8 @@ add_library(qserv_meta SHARED) target_sources(qserv_meta PRIVATE + JobStatus.cc + MessageStore.cc QMeta.cc QMetaMysql.cc QMetaSelect.cc @@ -10,7 +12,6 @@ target_sources(qserv_meta PRIVATE target_link_libraries(qserv_meta PUBLIC cconfig - qdisp qserv_common log mysqlclient_r diff --git a/src/qdisp/JobStatus.cc b/src/qmeta/JobStatus.cc similarity index 70% rename from src/qdisp/JobStatus.cc rename to src/qmeta/JobStatus.cc index 20dad135a3..3b5750ae3e 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,48 @@ #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); + /* &&& + 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::_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 +170,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 77% rename from src/qdisp/JobStatus.h rename to src/qmeta/JobStatus.h index d693921001..4e361844ac 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,13 @@ class JobStatus { UNKNOWN = 0, REQUEST = 1203, RESPONSE_READY, - RESPONSE_ERROR, RESPONSE_DATA, RESPONSE_DATA_NACK, RESPONSE_DONE, - RESULT_ERROR, - MERGE_ERROR, CANCEL, + RESPONSE_ERROR, // Errors must be between CANCEL and COMPLETE + RESULT_ERROR, // &&&uj ERRORS and CANCEL should probably be + MERGE_ERROR, // &&&uj separate from State, but may cause issues. COMPLETE = 2000 }; @@ -96,6 +96,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 +123,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 +145,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 0a215965ca..d4b066ecbc 100644 --- a/src/qmeta/QMeta.h +++ b/src/qmeta/QMeta.h @@ -38,12 +38,13 @@ #include "qmeta/types.h" namespace lsst::qserv::qdisp { -class MessageStore; class QueryMessage; } // namespace lsst::qserv::qdisp namespace lsst::qserv::qmeta { +class MessageStore; + /// @addtogroup qmeta /** @@ -328,7 +329,7 @@ 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. diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index 16a4206c78..c8095acff9 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -36,9 +36,9 @@ // Qserv headers #include "global/stringUtil.h" -#include "qdisp/JobStatus.h" -#include "qdisp/MessageStore.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" @@ -807,14 +807,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) { @@ -823,11 +823,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); } @@ -836,7 +836,7 @@ void QMetaMysql::addQueryMessages(QueryId queryId, shared_ptr QMetaMysql::_getChunkMapUpdateTime(lock } } -void QMetaMysql::_addQueryMessage(QueryId queryId, qdisp::QueryMessage const& qMsg, int& cancelCount, +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; @@ -981,7 +981,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 240d38a066..14809702c2 100644 --- a/src/qmeta/QMetaMysql.h +++ b/src/qmeta/QMetaMysql.h @@ -41,6 +41,8 @@ class SqlConnection; namespace lsst::qserv::qmeta { +class QueryMessage; + /// @addtogroup qmeta /** @@ -261,7 +263,7 @@ 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 = @@ -294,7 +296,7 @@ class QMetaMysql : public QMeta { 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 ccb9a47f23..02f6248923 100644 --- a/src/qmeta/testQMeta.cc +++ b/src/qmeta/testQMeta.cc @@ -33,6 +33,7 @@ #include "lsst/log/Log.h" // Qserv headers +#include "qmeta/MessageStore.h" #include "qmeta/QMetaMysql.h" #include "qmeta/QStatusMysql.h" #include "sql/SqlConnection.h" diff --git a/src/qproc/TaskMsgFactory.cc b/src/qproc/TaskMsgFactory.cc index 6302ead41b..64f26b28ec 100644 --- a/src/qproc/TaskMsgFactory.cc +++ b/src/qproc/TaskMsgFactory.cc @@ -296,9 +296,9 @@ void TaskMsgFactory::_addFragmentJson(nlohmann::json& jsFragments, std::string c LOGS(_log, LOG_LVL_WARN, __func__ << tmp); tmp = "&&&SUBCa scids="; - for (auto const& qstr : subchunkIds) { + for (auto const& scid : subchunkIds) { tmp += "~"; - tmp += qstr + "~"; + tmp += to_string(scid) + "~"; } LOGS(_log, LOG_LVL_WARN, __func__ << tmp); diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index af61bc01ad..357e2fafa7 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -218,7 +218,7 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { void InfileMerger::mergeCompleteFor(int jobId) { std::lock_guard resultSzLock(_mtxResultSizeMtx); - _totalResultSize += _perJobResultSize[jobId]; //&&&uj this can probably be simplified + _totalResultSize += _perJobResultSize[jobId]; //&&&uj this can probably be simplified } bool InfileMerger::merge(proto::ResponseSummary const& responseSummary, @@ -334,7 +334,7 @@ bool InfileMerger::merge(proto::ResponseSummary const& responseSummary, } bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { - auto jq = uberJob; // &&& replace jq with uberJob + auto jq = uberJob; // &&& replace jq with uberJob //&&&JobId const jobId = responseSummary.jobid(); UberJobId const uJobId = uberJob->getJobId(); //&&&std::string queryIdJobStr = QueryIdHelper::makeIdStr(responseSummary.queryid(), jobId); @@ -449,7 +449,6 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response 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 ef23ed1c11..5053207634 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -54,9 +54,11 @@ class ResponseSummary; } // namespace proto namespace qdisp { class JobQuery; -class MessageStore; class UberJob; } // namespace qdisp +namespace QMeta { +class MessageStore; +} namespace qproc { class DatabaseModels; } diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index ac26219bea..f35aab9e77 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -48,7 +48,6 @@ #include "util/TimeUtils.h" #include "xrdsvc/StreamBuffer.h" - // LSST headers #include "lsst/log/Log.h" @@ -285,15 +284,16 @@ FileChannelShared::FileChannelShared(shared_ptr const& sendC } //&&&uj -FileChannelShared::Ptr FileChannelShared::create(std::shared_ptr const& uberJob, qmeta::CzarId czarId, - string const& czarHostName, int czarPort, - string const& workerId) { +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) +FileChannelShared::FileChannelShared(std::shared_ptr const& uberJobData, + qmeta::CzarId czarId, string const& czarHostName, int czarPort, + string const& workerId) : _isUberJob(true), _sendChannel(nullptr), _uberJobData(uberJobData), @@ -305,11 +305,13 @@ FileChannelShared::FileChannelShared(std::shared_ptr const& _protobufArena(make_unique()), _scsId(scsSeqId++), _useHttp(true) { - LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created"); - LOGS(_log, LOG_LVL_WARN, "&&& FileChannelShared created useHttp true"); + LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created scsId=" << _scsId); + LOGS(_log, LOG_LVL_WARN, + "&&& FileChannelShared created useHttp true scsId=" << _scsId << " ujId=" << _uberJobId); } FileChannelShared::~FileChannelShared() { + LOGS(_log, LOG_LVL_WARN, "&&& ~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 @@ -360,6 +362,8 @@ string FileChannelShared::makeIdStr(int qId, int jId) { bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared_ptr const& task, bool cancelled) { + LOGS(_log, LOG_LVL_WARN, + "&&& FileChannelShared::buildAndTransmitError scsId=" << _scsId << " ujId=" << _uberJobId); lock_guard const tMtxLock(_tMtx); if (!_useHttp) { if (!_sendResponse(tMtxLock, task, cancelled, multiErr)) { @@ -368,7 +372,16 @@ bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared } return true; } else { - LOGS(_log, LOG_LVL_WARN, "&&&uj NEED CODE send msg to czar with the errors"); + LOGS(_log, LOG_LVL_WARN, "&&&uj send msg to czar with the errors"); + 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 false; } @@ -437,6 +450,8 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& streamMutexLock, string c LOGS(_log, LOG_LVL_WARN, "FileChannelShared first kill call " << note); } // &&&uj anything else need to be done? + _removeFile(streamMutexLock); return oldVal; } } @@ -499,6 +515,7 @@ bool FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p LOGS(_log, LOG_LVL_TRACE, __func__ << " _fillRows " << task->getIdStr() << " end"); _responseData->set_rowcount(rows); _responseData->set_transmitsize(tSize); + ++_headerCount; // Serialize the content of the data buffer into the Protobuf data message // that will be written into the output file. @@ -509,7 +526,10 @@ bool FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p LOGS(_log, LOG_LVL_TRACE, __func__ << " file write " << task->getIdStr() << " start"); // Create the file if not open. if (!_file.is_open()) { + auto oldname = _fileName; // &&& del _fileName = task->resultFilePath(); + LOGS(_log, LOG_LVL_WARN, + "&&& opening file oldName=" << oldname << " new=" << _fileName << " scsId=" << _scsId); _file.open(_fileName, ios::out | ios::trunc | ios::binary); if (!(_file.is_open() && _file.good())) { throw runtime_error("FileChannelShared::" + string(__func__) + @@ -523,6 +543,7 @@ bool FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p uint32_t const msgSizeBytes = msg.size(); _file.write(reinterpret_cast(&msgSizeBytes), sizeof msgSizeBytes); _file.write(msg.data(), msgSizeBytes); + LOGS(_log, LOG_LVL_WARN, "&&&uj headerCount=" << _headerCount << " wrote msgSizeBytes=" << msgSizeBytes); if (!(_file.is_open() && _file.good())) { throw runtime_error("FileChannelShared::" + string(__func__) + " failed to write " + @@ -562,21 +583,28 @@ bool FileChannelShared::_fillRows(lock_guard const& tMtxLock, MYSQL_RES* } void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { - if (!_fileName.empty() && _file.is_open()) { - _file.close(); + LOGS(_log, LOG_LVL_WARN, "&&& 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, bool cancelled, util::MultiError const& multiErr) { - LOGS(_log, LOG_LVL_WARN, "&&& FileChannelShared::_sendResponse"); + LOGS(_log, LOG_LVL_WARN, + "&&& FileChannelShared::_sendResponse scsId=" << _scsId << " task=" << task->getIdStr()); auto const queryId = task->getQueryId(); auto const jobId = task->getJobId(); auto const idStr(makeIdStr(queryId, jobId)); @@ -602,40 +630,41 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ // Prepare the response object and serialize in into a message that will // be sent to Czar. + 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); - 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()); - 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) { + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&& idStr=" << idStr << ", _responseBuf.size()=" << _responseBuf.size() + << " useHttp=" << _useHttp); + //&&&if (!_useHttp) { // 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); @@ -652,18 +681,20 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ return false; } } else { - LOGS(_log, LOG_LVL_WARN, "&&&uj NEED CODE send the url back with http"); + LOGS(_log, LOG_LVL_WARN, + "&&&uj send the url back with http scsId=" << _scsId << " task=" << task->getIdStr()); // &&&uj the http communications need to happen in a different thread, or this thread can be booted - // from - // &&&uj the scheduler so that it can just wait for a response. + // &&&uj from the scheduler so that it can just wait for a response. 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(); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj ujId="<< _uberJobId << " httpFileUrl=" << httpFileUrl); - ujData->fileReadyResponse(httpFileUrl, _rowcount, _transmitsize); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&uj ujId=" << _uberJobId << " scsId=" << _scsId << " task=" << task->getIdStr() + << " httpFileUrl=" << httpFileUrl); + ujData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); } return true; } diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 835a857cef..8d6069fc8c 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -134,9 +134,9 @@ class FileChannelShared { ComInfoToCzar::Ptr const& comInfoToCzar); */ /// 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); //&&& delete all params except uberJob - + static Ptr create(std::shared_ptr const& uberJob, qmeta::CzarId czarId, + std::string const& czarHostName, int czarPort, + std::string const& workerId); //&&& delete all params except uberJob FileChannelShared() = delete; FileChannelShared(FileChannelShared const&) = delete; @@ -190,8 +190,8 @@ class FileChannelShared { FileChannelShared(std::shared_ptr const& sendChannel, qmeta::CzarId czarId, std::string const& workerId); - FileChannelShared(std::shared_ptr const& uberJob, qmeta::CzarId czarId, std::string const& czarHostName, - int czarPort, std::string const& workerId); + 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. @@ -259,7 +259,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::weak_ptr _uberJobData; ///< &&& doc + std::weak_ptr _uberJobData; ///< &&& doc UberJobId const _uberJobId; ///< &&& doc qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). &&& delete @@ -305,6 +305,7 @@ 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; ///< &&& del //&&&ComInfoToCzar::Ptr _comInfoToCzar; ///< &&&uj doc diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index ba4d1ee610..3dd1b97af8 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -81,7 +81,7 @@ string buildResultFilePath(shared_ptr const& taskMs return path.string(); } -std::atomic fileSeq = 0; +// std::atomic fileSeq = 0; &&& //&&&uj string buildUjResultFilePath(lsst::qserv::wbase::UberJobData::Ptr const& ujData, string const& resultsDirname) { @@ -90,7 +90,7 @@ string buildUjResultFilePath(lsst::qserv::wbase::UberJobData::Ptr const& ujData, // 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()) + "-" + to_string(fileSeq++) + ".proto"; + to_string(ujData->getUberJobId()) + "-0" + ".proto"; return path.string(); } diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index c2e5df7482..f4ade93149 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -37,6 +37,7 @@ #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" @@ -52,8 +53,8 @@ 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, +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), @@ -63,7 +64,8 @@ UberJobData::UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta _queryId(queryId), _workerId(workerId), _authKey(authKey), - _foreman(foreman) {} + _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) { @@ -72,8 +74,12 @@ void UberJobData::setFileChannelShared(std::shared_ptr const& _fileChannelShared = fileChannelShared; } -void UberJobData::fileReadyResponse(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize) { - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse a httpFileUrl=" << httpFileUrl); +void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, + uint64_t headerCount) { + string const funcN = getFuncIdStr(__func__); + LOGS(_log, LOG_LVL_WARN, + funcN << "&&& UberJobData::responseFileReady a httpFileUrl=" << httpFileUrl << " rows=" << rowCount + << " fSize=" << fileSize << " headerCount=" << headerCount); json request = {{"version", http::MetaModule::version}, {"workerid", _foreman->chunkInventory()->id()}, @@ -84,49 +90,112 @@ void UberJobData::fileReadyResponse(string const& httpFileUrl, uint64_t rowCount {"uberjobid", _uberJobId}, {"fileUrl", httpFileUrl}, {"rowCount", rowCount}, - {"fileSize", fileSize}}; + {"fileSize", fileSize}, + {"headerCount", headerCount}}; - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse b"); + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseFileReady b"); auto const method = http::Method::POST; vector const headers = {"Content-Type: application/json"}; //&&&string const url = "http://" + _czarName + ":" + to_string(_czarPort) + "/queryjob-ready"; string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-ready"; string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse c"); - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJobData::fileReadyResponse url=" << url << " request=" << request.dump()); + LOGS(_log, LOG_LVL_WARN, + funcN + "&&&uj UberJobData::responseFileReady url=" << url << " request=" << request.dump()); http::Client client(method, url, request.dump(), headers); - int maxTries = 2; // &&& set from config + int maxTries = 2; // &&& set from config bool transmitSuccess = false; - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse d"); - for (int j=0; j()) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj success"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj success"); transmitSuccess = true; } else { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE success=0"); + LOGS(_log, LOG_LVL_WARN, + funcN << "&&&uj NEED CODE success=0, result file should probably be deleted."); + 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, requestContext + " &&&uj failed, ex: " + ex.what()); + LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " &&&uj failed, ex: " + ex.what()); } } - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse e"); if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, - __func__ << "&&&uj NEED CODE Let czar find out through polling worker status???"); - } else { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); + funcN << "&&&uj NEED CODE Let czar find out through polling worker status??? Just throw the " + "result away???"); } - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::fileReadyResponse end"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&& UberJobData::responseFileReady end"); } +bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& task, + bool cancelled) { + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseError a"); + string const funcN = getFuncIdStr(__func__); + + 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}}; + + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseError b"); + + auto const method = http::Method::POST; + vector const headers = {"Content-Type: application/json"}; + //&&&string const url = "http://" + _czarName + ":" + to_string(_czarPort) + "/queryjob-ready"; + string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-error"; + string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; + LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseError c"); + LOGS(_log, LOG_LVL_WARN, + funcN << "&&&uj UberJobData::responseError url=" << url << " request=" << request.dump()); + http::Client client(method, url, request.dump(), headers); + int maxTries = 2; // &&& set from config + bool transmitSuccess = false; + for (int j = 0; !transmitSuccess && j < maxTries; ++j) { + try { + json const response = client.readAsJson(); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj response=" << response); + if (0 != response.at("success").get()) { + LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj success"); + transmitSuccess = true; + } else { + LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj NEED CODE 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 + " &&&uj failed, ex: " + ex.what()); + } + } + return transmitSuccess; +} } // namespace lsst::qserv::wbase diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index 642f12b865..565928f40d 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -37,9 +37,16 @@ #include "qmeta/types.h" #include "wbase/SendChannel.h" -namespace lsst::qserv::wcontrol { +namespace lsst::qserv { + +namespace util { +class MultiError; +} + +namespace wcontrol { class Foreman; } +} // namespace lsst::qserv namespace lsst::qserv::wbase { @@ -55,10 +62,12 @@ class UberJobData { 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)); + 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)); } // &&& doc void setFileChannelShared(std::shared_ptr const& fileChannelShared); @@ -76,11 +85,20 @@ class UberJobData { } /// &&& doc - void fileReadyResponse(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize); + void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, + uint64_t headerCount); // &&& remove headerCount + + /// &&& doc + bool responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); + + std::string getIdStr() const { return _idStr; } + std::string getFuncIdStr(std::string const& funcName) { + return getIdStr() + " UberJobData::" + funcName + " "; + } private: - UberJobData(UberJobId uberJobId, std::string const& czarName, qmeta::CzarId czarId, std::string czarHost, int czarPort, - uint64_t queryId, std::string const& workerId, + 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; @@ -106,7 +124,7 @@ class UberJobData { //&&& uint64_t const queryId; //&&&uint64_t const uberJobId; - + std::string const _idStr; }; } // namespace lsst::qserv::wbase diff --git a/src/wbase/UserQueryInfo.h b/src/wbase/UserQueryInfo.h index b23865bb29..5299985a94 100644 --- a/src/wbase/UserQueryInfo.h +++ b/src/wbase/UserQueryInfo.h @@ -82,7 +82,7 @@ class UserQueryInfo { /// Map of all UberJobData objects on this worker for this User Query. std::map> _uberJobMap; - std::mutex _uberJobMapMtx; ///< protects _uberJobMap; + std::mutex _uberJobMapMtx; ///< protects _uberJobMap; }; } // namespace lsst::qserv::wbase diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index 4303fa6e2d..088ad8d18e 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -72,7 +72,7 @@ void ChunkTasksQueue::queueTask(std::vector const& tasks) { void ChunkTasksQueue::queueTask(std::vector const& tasks) { std::lock_guard lg(_mapMx); auto iter = _chunkMap.end(); - int prevChunkId = -1; // invalid chunkId number + int prevChunkId = -1; // invalid chunkId number for (auto const& task : tasks) { int chunkId = task->getChunkId(); /* &&& diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index dd182ddc07..e4410a1d62 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -234,7 +234,6 @@ void ScanScheduler::queCmd(vector const& cmds) { jid = tsk->getJobId(); QSERV_LOGCONTEXT_QUERY_JOB(qid, jid); } else { - // &&&if (qid != t->getQueryId() || jid != t->getJobId()) { if (qid != tsk->getQueryId()) { string eMsg("Mismatch multiple query/job ids in single queCmd "); diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 78b5727396..3c633cf7af 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -146,8 +146,8 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { __func__ << "&&&SUBC uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); //&&&uj make UberJobData, UseQueryInfo entry, FileChannelShared, and Tasks. - auto ujData = - wbase::UberJobData::create(ujId, czarName, czarId, czarHostName, czarPort, ujQueryId, targetWorkerId, foreman(), authKey()); + auto ujData = wbase::UberJobData::create(ujId, czarName, czarId, czarHostName, czarPort, ujQueryId, + targetWorkerId, foreman(), authKey()); LOGS(_log, LOG_LVL_WARN, "&&&uj (ujData != nullptr) = " << (ujData != nullptr)); // Find the entry for this queryId, creat a new one if needed. @@ -158,7 +158,6 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { wbase::FileChannelShared::create(ujData, czarId, czarHostName, czarPort, targetWorkerId); ujData->setFileChannelShared(channelShared); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); for (auto const& job : ujJobs) { json const& jsJobDesc = job["jobdesc"]; @@ -230,7 +229,8 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { timer.start(); foreman()->processTasks(ujTasks); // Queues tasks to be run later. timer.stop(); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&& Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&& Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); #if 0 /// &&&&&&&& // Now that the request is decoded (successfully or not), release the @@ -264,6 +264,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { return jsRet; } +#if 0 //&&& // &&&uj delete void HttpWorkerCzarModule::_temporaryRespFunc(string const& targetWorkerId, string const& czarName, qmeta::CzarId czarId, string const& czarHostName, int czarPort, @@ -303,5 +304,6 @@ void HttpWorkerCzarModule::_temporaryRespFunc(string const& targetWorkerId, stri LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); } } +#endif // &&& } // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/xrdsvc/HttpWorkerCzarModule.h index 2f0ef10723..6be1dd4f1f 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.h +++ b/src/xrdsvc/HttpWorkerCzarModule.h @@ -81,11 +81,13 @@ class HttpWorkerCzarModule : public xrdsvc::HttpModule { /// &&& doc nlohmann::json _handleQueryJob(std::string const& func); + /* &&& /// &&&uj temporary function for testing communication. Something like this will /// need to be called when the uberjob has finished making the result file. void _temporaryRespFunc(std::string const& targetWorkerId, std::string const& czarName, qmeta::CzarId czarId, std::string const& czarHostName, int czarPort, uint64_t ujQueryId, uint64_t ujId); + */ }; } // namespace lsst::qserv::xrdsvc From 9b70fcad5e6e159422b8d3321f1937d69ebd96e6 Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 27 Jun 2024 16:28:57 -0700 Subject: [PATCH 06/15] Errors reported correctly. --- src/ccontrol/MergingHandler.cc | 16 ++++++++++- src/ccontrol/MergingHandler.h | 3 +- src/ccontrol/UserQuerySelect.cc | 5 ++-- src/czar/Czar.cc | 10 +++++++ src/qdisp/Executive.cc | 50 ++++++++++++++++----------------- src/qdisp/Executive.h | 18 +++++++++--- src/qdisp/JobQuery.cc | 5 +++- src/qdisp/ResponseHandler.h | 3 ++ src/qdisp/UberJob.cc | 17 +++++++---- 9 files changed, 87 insertions(+), 40 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index c0efa698bc..77eba7b84c 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -588,7 +588,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& responseSummary, uint32_t& resultRows) { _wName = responseSummary.wname(); @@ -805,4 +805,18 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe return {success, shouldCancel}; } +void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int status) { + /* &&& + _error = util::Error(responseSummary.errorcode(), responseSummary.errormsg(), + util::ErrorCode::MYSQLEXEC); + _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); + _flushError(jq); + */ + + 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 ca3a4b7f03..2a228c88f7 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -84,7 +84,7 @@ class MergingHandler : public qdisp::ResponseHandler { uint64_t& resultRows) override; /// &&&uj doc - bool 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; @@ -130,6 +130,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}; ///< &&& doc 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/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 1e804a6cb7..c034ff530e 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -881,16 +881,15 @@ void UserQuerySelect::discard() { } } - // Remove _executive from ExecutiveMap &&&uj - _executive->removeFromMap(); - // Make sure resources are released. if (_executive && _executive->getNumInflight() > 0) { throw UserQueryError(getQueryIdString() + " Executive unfinished, cannot discard"); } + _executive.reset(); _messageStore.reset(); _qSession.reset(); + try { _discardMerger(); } catch (UserQueryError const& e) { diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 0eaf872abe..843909ce83 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -337,19 +337,29 @@ 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()); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " a"); // Add logging context with query ID QSERV_LOGCONTEXT_QUERY(uq->getQueryId()); LOGS(_log, LOG_LVL_DEBUG, "submitting new query"); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " b"); uq->submit(); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " c"); uq->join(); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " d"); try { msgTable.unlock(uq); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " e"); if (uq) uq->discard(); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " f"); } catch (std::exception const& exc) { // TODO? if this fails there is no way to notify client, and client // will likely hang because table may still be locked. LOGS(_log, LOG_LVL_ERROR, "Query finalization failed (client likely hangs): " << exc.what()); } + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " g"); + uq.reset(); + LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " end"); }; LOGS(_log, LOG_LVL_DEBUG, "starting finalizer thread for query"); thread finalThread(finalizer); diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index a752b57508..ed7955b240 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -101,7 +101,7 @@ string getErrorText(XrdSsiErrInfo& e) { namespace lsst::qserv::qdisp { mutex Executive::_executiveMapMtx; ///< protects _executiveMap -map Executive::_executiveMap; ///< Map of executives for queries in progress. +map> Executive::_executiveMap; ///< Map of executives for queries in progress. //////////////////////////////////////////////////////////////////////// // class Executive implementation @@ -124,12 +124,17 @@ Executive::~Executive() { LOGS(_log, LOG_LVL_WARN, "&&& Executive::~Executive() " << getIdStr()); qdisp::CzarStats::get()->deleteQuery(); qdisp::CzarStats::get()->deleteJobs(_incompleteJobs.size()); + // Remove this executive from the map. + if (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(); qdisp::CzarStats::get()->untrackQueryProgress(_id); } + LOGS(_log, LOG_LVL_WARN, "&&& Executive::~Executive() " << getIdStr() << " end"); } Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptr const& ms, @@ -190,25 +195,17 @@ void Executive::setQueryId(QueryId id) { qdisp::CzarStats::get()->trackQueryProgress(_id); } -void Executive::removeFromMap() { - if (_queryIdSet) { - return; - } - // Remove this from the global executive map. - lock_guard lgMap(_executiveMapMtx); - auto iter = _executiveMap.find(_id); - if (iter != _executiveMap.end()) { - _executiveMap.erase(iter); - } -} - Executive::Ptr Executive::getExecutiveFromMap(QueryId qId) { lock_guard lgMap(_executiveMapMtx); auto iter = _executiveMap.find(qId); if (iter == _executiveMap.end()) { return nullptr; } - return iter->second; + Executive::Ptr exec = iter->second.lock(); + if (exec == nullptr) { + _executiveMap.erase(iter); + } + return exec; } UberJob::Ptr Executive::findUberJob(UberJobId ujId) { @@ -262,14 +259,7 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { } QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getJobId()); - /* &&&uj - //&&&uj code just returns the jobQuery at this point, it doesn't call runJob(). - LOGS(_log, LOG_LVL_DEBUG, "Executive::add with path=" << jobDesc->resource().path()); - bool started = jobQuery->runJob(); - if (!started && isLimitRowComplete()) { - markCompleted(jobQuery->getIdInt(), false); - } - */ + LOGS(_log, LOG_LVL_WARN, "&&& Executive::add end"); return jobQuery; } @@ -436,6 +426,16 @@ JobQuery::Ptr Executive::getSharedPtrForRawJobPtr(JobQuery* jqRaw) { //&&& return jq; } +/// &&& doc +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. /// @@ -794,8 +794,8 @@ void Executive::_waitAllUntilEmpty() { void Executive::_addToChunkJobMap(JobQuery::Ptr const& job) { int chunkId = job->getDescription()->resource().chunk(); - auto entry = pair(chunkId, job.get()); - // LOGS(_log, LOG_LVL_WARN, "&&& _addToChunkJobMap chunkId=" << chunkId); + auto entry = pair(chunkId, job); + // LOGS(_log, LOG_LVL_WARN, "&&& _addToChunkJobMap chunkId=" << chunkId); &&& lock_guard lck(_chunkToJobMapMtx); if (_chunkToJobMapInvalid) { throw util::Bug(ERR_LOC, "&&& map insert FAILED, map is already invalid"); @@ -846,7 +846,7 @@ ostream& operator<<(ostream& os, Executive::JobMap::value_type const& v) { 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 c413ccf7ee..764f4434ab 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -104,7 +104,6 @@ class Executive : public std::enable_shared_from_this { typedef std::shared_ptr Ptr; typedef std::unordered_map> JobMap; typedef int ChunkIdType; //&&&uj This type is probably not needed - //&&&typedef std::map ChunkIdJobMapType; typedef std::map> ChunkIdJobMapType; /// Construct an Executive. @@ -121,6 +120,10 @@ class Executive : public std::enable_shared_from_this { /// &&& doc static Ptr getExecutiveFromMap(QueryId qId); + std::string cName(const char* funcName="") { + return std::string("Executive::") + funcName; + } + /// &&&uj doc void setUserQuerySelect(std::shared_ptr const& uqs) { _userQuerySelect = uqs; } //&&&void buildAndSendUberJobs(int const maxChunksPerUber); @@ -128,8 +131,10 @@ class Executive : public std::enable_shared_from_this { /// &&&uj doc Return a map that only contains Jobs not assigned to an UberJob. ChunkIdJobMapType unassignedChunksInQuery(); + /* &&& /// &&& doc void removeFromMap(); + */ /// &&& doc std::shared_ptr findUberJob(UberJobId ujId); @@ -220,16 +225,21 @@ class Executive : public std::enable_shared_from_this { /// &&&uj doc void assignJobsToUberJobs(); - ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& delete + //&&& ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& delete bool startUberJob(std::shared_ptr const& uJob); /// &&& - std::shared_ptr getSharedPtrForRawJobPtr(JobQuery* jqRaw); /// &&& + std::shared_ptr getSharedPtrForRawJobPtr(JobQuery* jqRaw); /// &&& delete int getTotalJobs() { return _totalJobs; } void setFlagFailedUberJob(bool val) { _failedUberJob = val; } + /// &&& doc + void addMultiError(int errorCode, std::string const& errorMsg, int errState); + std::string dumpUberJobCounts() const; + + private: Executive(ExecutiveConfig const& c, std::shared_ptr const& ms, SharedResources::Ptr const& sharedResources, std::shared_ptr const& qStatus, @@ -357,7 +367,7 @@ class Executive : public std::enable_shared_from_this { std::atomic _failedUberJob{false}; static std::mutex _executiveMapMtx; ///< protects _executiveMap - static std::map _executiveMap; ///< Map of executives for queries in progress. + static std::map> _executiveMap; ///< Map of executives for queries in progress. }; /// &&&uj MarkCompleteFunc is not needed with uberjobs. diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index aa4deaa0f5..3debfcfdec 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -58,7 +58,10 @@ 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"); } +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. diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index d549947025..5f2cf1d308 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -74,6 +74,9 @@ class ResponseHandler { virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, uint64_t& resultRows) = 0; + /// &&& doc + 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 3d3b81c1a0..cc6c536d68 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -303,7 +303,6 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ "&&&uj UberJob::importResultFile fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); - // It's possible jq and _jobQuery differ, so need to use jq. if (isQueryCancelled()) { LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile import job was cancelled."); return _importResultError(true, "cancelled", "Query cancelled"); @@ -384,9 +383,10 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { LOGS(_log, LOG_LVL_INFO, "UberJob::workerError code=" << errorCode << " msg=" << errorMsg); //&&&uj NEED CODE update status for each job in this uberjob - // jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); + //jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); bool const deleteData = true; + bool const keepData = !deleteData; auto exec = _executive.lock(); if (exec == nullptr || isQueryCancelled()) { LOGS(_log, LOG_LVL_WARN, "UberJob::workerError no executive or cancelled"); @@ -401,15 +401,22 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { "UberJob ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } - return _workerErrorFinish(!deleteData, "none", "limitRowComplete"); + return _workerErrorFinish(keepData, "none", "limitRowComplete"); } LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError d"); - //&&&uj get error message to the user? - + /* &&& JobBase::Ptr jBaseThis = shared_from_this(); weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); + */ + + //&&&uj get error message to the user and kill the user query. + //&&&jq->getDescription()->respHandler()->flushHttpError(errorCode, errorMsg, util::ErrorCode::MYSQLEXEC); + 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, ""); From 9df25313e13d38c0adff0d7000fb5481dae81c2e Mon Sep 17 00:00:00 2001 From: John Gates Date: Fri, 28 Jun 2024 17:28:28 -0700 Subject: [PATCH 07/15] Fails appropriately from max UberJob attempts. --- src/czar/Czar.cc | 70 +++++++++++++++++- src/czar/Czar.h | 23 ++++++ src/czar/CzarChunkMap.cc | 10 +++ src/czar/CzarChunkMap.h | 6 ++ src/czar/HttpCzarWorkerModule.cc | 5 +- src/qdisp/Executive.cc | 35 ++++++--- src/qdisp/Executive.h | 13 ++-- src/qdisp/JobDescription.cc | 2 +- src/qdisp/JobDescription.h | 1 + src/qdisp/JobQuery.cc | 9 ++- src/qdisp/JobQuery.h | 7 +- src/qdisp/UberJob.cc | 119 ++++++++++++++++--------------- src/qdisp/UberJob.h | 9 +-- src/qmeta/JobStatus.h | 5 +- 14 files changed, 231 insertions(+), 83 deletions(-) diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 843909ce83..01c083753a 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -54,6 +54,7 @@ #include "http/Method.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" @@ -167,6 +168,45 @@ Czar::Ptr Czar::createCzar(string const& configFilePath, string const& czarName) return _czar; } +void Czar::_monitor() { + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor a"); + while(_monitorLoop) { + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor b"); + this_thread::sleep_for(_monitorSleepTime); + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor c"); + + + /// Check database for changes in worker chunk assignments and aliveness + _czarChunkMap->read(); + + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor d"); + /// Create new UberJobs for all jobs that got 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; + } + } + } + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor e"); + // Use the copy to create new UberJobs as needed + for (auto&& [qIdKey, execVal] : execMap) { + execVal->assignJobsToUberJobs(); + } + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor f"); + } + LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor end"); +} + // Constructors Czar::Czar(string const& configFilePath, string const& czarName) : _czarName(czarName), @@ -263,9 +303,19 @@ Czar::Czar(string const& configFilePath, string const& czarName) _czarConfig->setReplicationHttpPort(port); _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()"); } +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)); @@ -699,4 +749,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 f279acd583..9e110d9ff9 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -59,6 +59,11 @@ 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 CzarChunkMap; @@ -133,6 +138,12 @@ class Czar { std::shared_ptr getCzarRegistry() const { return _czarRegistry; } + /// &&& doc + void insertExecutive(QueryId qId, std::shared_ptr const& execPtr); + + /// &&& doc + std::shared_ptr getExecutiveFromMap(QueryId qId); + private: /// Private constructor for singleton. Czar(std::string const& configFilePath, std::string const& czarName); @@ -152,6 +163,9 @@ class Czar { /// @return An identifier of the last query that was recorded in the query metadata table QueryId _lastQueryIdBeforeRestart() const; + /// &&& doc + void _monitor(); + static Ptr _czar; ///< Pointer to single instance of the Czar. // combines client name (ID) and its thread ID into one unique ID @@ -193,6 +207,15 @@ class Czar { /// 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; ///< &&& doc + std::atomic _monitorLoop{true}; ///< &&& doc + std::chrono::milliseconds _monitorSleepTime{15000}; ///< Wait time between checks. &&& set from config + }; } // namespace lsst::qserv::czar diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index fac5d6eb35..5c36e466d3 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -59,6 +59,16 @@ CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta( CzarChunkMap::~CzarChunkMap() { LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); } +bool CzarChunkMap::read() { + bool mapsSet = false; + try { + mapsSet = _read(); + } catch (qmeta::QMetaError const& qExc) { + LOGS(_log, LOG_LVL_ERROR, __func__ << " CzarChunkMap could not read DB " << qExc.what()); + } + return mapsSet; +} + bool CzarChunkMap::_read() { LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_read() start"); // If replacing the map, this may take a bit of time, but it's probably diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index d4828c0613..c3e5fa432c 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -48,6 +48,9 @@ class ChunkMapException : public util::Issue { ChunkMapException(Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} }; +// &&&&&&&&&& Provide a map based on family name (which will be based on database name for now) +// to determine which maps to use. 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. @@ -216,6 +219,9 @@ class CzarChunkMap { return {_chunkMap, _workerChunkMap}; } + /// &&& doc + bool read(); + private: /// Try to `_read` values for maps from `qmeta`. CzarChunkMap(std::shared_ptr const& qmeta); diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index d722d37042..a76add0ba1 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -28,6 +28,7 @@ // Qserv headers #include "cconfig/CzarConfig.h" +#include "czar/Czar.h" #include "qdisp/Executive.h" #include "qdisp/UberJob.h" #include "global/intTypes.h" @@ -117,7 +118,7 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR errorMsg=" << errorMsg); // Find UberJob - qdisp::Executive::Ptr exec = qdisp::Executive::getExecutiveFromMap(queryId); + 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)); @@ -167,7 +168,7 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR fileSize=" << fileSize); // Find UberJob - qdisp::Executive::Ptr exec = qdisp::Executive::getExecutiveFromMap(queryId); + 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)); diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index ed7955b240..f6549fb828 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -100,8 +100,10 @@ string getErrorText(XrdSsiErrInfo& e) { namespace lsst::qserv::qdisp { +/* &&& mutex Executive::_executiveMapMtx; ///< protects _executiveMap map> Executive::_executiveMap; ///< Map of executives for queries in progress. +*/ //////////////////////////////////////////////////////////////////////// // class Executive implementation @@ -125,7 +127,7 @@ Executive::~Executive() { qdisp::CzarStats::get()->deleteQuery(); qdisp::CzarStats::get()->deleteJobs(_incompleteJobs.size()); // Remove this executive from the map. - if (getExecutiveFromMap(getId()) != nullptr) { + 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. @@ -155,6 +157,8 @@ 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 { @@ -189,12 +193,16 @@ void Executive::setQueryId(QueryId id) { // Insert into the global executive map. { + /* &&& lock_guard lgMap(_executiveMapMtx); _executiveMap[_id] = shared_from_this(); + */ + czar::Czar::getCzar()->insertExecutive(_id, shared_from_this()); } qdisp::CzarStats::get()->trackQueryProgress(_id); } +/* &&& Executive::Ptr Executive::getExecutiveFromMap(QueryId qId) { lock_guard lgMap(_executiveMapMtx); auto iter = _executiveMap.find(qId); @@ -207,6 +215,7 @@ Executive::Ptr Executive::getExecutiveFromMap(QueryId qId) { } return exec; } +*/ UberJob::Ptr Executive::findUberJob(UberJobId ujId) { lock_guard lgMap(_uberJobsMapMtx); @@ -295,15 +304,26 @@ void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect end"); } -/// &&&uj &&&&&&&&&&&&&&&&&&&&&& NEED CODE put this as command in qdisppool. + void Executive::runUberJob(std::shared_ptr const& uberJob) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); - bool started = uberJob->runUberJob(); - /* &&&uj - if (!started && isLimitRowComplete()) { - uberJob->callMarkCompleteFunc(false); + + bool const useqdisppool = true; /// &&& delete + if (useqdisppool) { + auto runUberJobFunc = [uberJob](util::CmdData*) { + LOGS(_log, LOG_LVL_WARN, "&&&uj Executive::runUberJob::runUberJobFunc a"); + uberJob->runUberJob(); + }; + + auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); + if (_scanInteractive) { + _qdispPool->queCmd(cmd, 0); + } else { + _qdispPool->queCmd(cmd, 1); + } + } else { + uberJob->runUberJob(); } - */ LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob end"); } @@ -397,7 +417,6 @@ string Executive::dumpUberJobCounts() const { void Executive::assignJobsToUberJobs() { auto uqs = _userQuerySelect.lock(); - /// &&& NEED CODE put on qdisppool &&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&&& uqs->buildAndSendUberJobs(); } diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 764f4434ab..2a6c560468 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -117,8 +117,10 @@ class Executive : public std::enable_shared_from_this { ~Executive(); + /* &&& /// &&& doc static Ptr getExecutiveFromMap(QueryId qId); + */ std::string cName(const char* funcName="") { return std::string("Executive::") + funcName; @@ -131,11 +133,6 @@ class Executive : public std::enable_shared_from_this { /// &&&uj doc Return a map that only contains Jobs not assigned to an UberJob. ChunkIdJobMapType unassignedChunksInQuery(); - /* &&& - /// &&& doc - void removeFromMap(); - */ - /// &&& doc std::shared_ptr findUberJob(UberJobId ujId); @@ -238,7 +235,9 @@ class Executive : public std::enable_shared_from_this { 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 private: Executive(ExecutiveConfig const& c, std::shared_ptr const& ms, @@ -366,8 +365,10 @@ class Executive : public std::enable_shared_from_this { /// executive can make new uberjobs. std::atomic _failedUberJob{false}; + /* &&& static std::mutex _executiveMapMtx; ///< protects _executiveMap static std::map> _executiveMap; ///< Map of executives for queries in progress. + */ }; /// &&&uj MarkCompleteFunc is not needed with uberjobs. diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 86efcea1ab..2a31278971 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -92,7 +92,7 @@ bool JobDescription::incrAttemptCountScrubResultsJson() { #endif // &&& ++_attemptCount; - if (_attemptCount > MAX_JOB_ATTEMPTS) { + if (_attemptCount >= MAX_JOB_ATTEMPTS) { LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); return false; } diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 2ec0d6cacf..a6ec53cbf7 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -107,6 +107,7 @@ class JobDescription { std::shared_ptr const& taskMsgFactory, std::shared_ptr const& chunkQuerySpec, std::string const& chunkResultName, bool mock = false); + qmeta::CzarId _czarId; QueryId _queryId; JobId _jobId; ///< Job's Id number. diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 3debfcfdec..9a0db86835 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -86,7 +86,7 @@ bool JobQuery::runJob() { // &&& LOGS(_log, LOG_LVL_DEBUG, "runJob checking attempt=" << _jobDescription->getAttemptCount()); lock_guard lock(_rmutex); - if (_jobDescription->getAttemptCount() < _getMaxAttempts()) { + if (_jobDescription->getAttemptCount() < executive->getMaxAttempts()) { bool okCount = _jobDescription->incrAttemptCountScrubResults(); if (!okCount) { criticalErr("hit structural max of retries"); @@ -179,7 +179,7 @@ bool JobQuery::isQueryCancelled() { bool JobQuery::_setUberJobId(UberJobId ujId) { QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); if (_uberJobId >= 0 && ujId != _uberJobId) { - LOGS(_log, LOG_LVL_ERROR, + LOGS(_log, LOG_LVL_DEBUG, __func__ << " couldn't change UberJobId as ujId=" << ujId << " is owned by " << _uberJobId); return false; } @@ -204,6 +204,11 @@ bool JobQuery::unassignFromUberJob(UberJobId ujId) { 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); } diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 0a62eafd07..76e5fd4e08 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -117,6 +117,8 @@ class JobQuery : public JobBase { return _isInUberJob(); } + int getAttemptCount() const; + /// If ujId is the current owner, clear ownership. /// @return true if job is unassigned. bool unassignFromUberJob(UberJobId ujId); @@ -127,13 +129,16 @@ class JobQuery : public JobBase { _jobDescription->respHandler()->setJobQuery(jbPtr); } + /* &&& /// NOTE: _rmutex must be held before calling this int _getRunAttemptsCount() const { std::lock_guard lock(_rmutex); return _jobDescription->getAttemptCount(); } + 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 _getAttemptSleepSeconds() const { return 15; } // As above or until added to config file. + */ /// @return true if _uberJobId was set, it can only be set if it is unassigned /// or by the current owner. diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index cc6c536d68..0b0586f8f0 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -38,6 +38,7 @@ #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" @@ -95,10 +96,9 @@ bool UberJob::addJob(JobQuery::Ptr const& job) { } bool UberJob::runUberJob() { - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() start"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << "&&& UberJob::runUberJob() start"); // &&&uj most, if not all, of this should be done in a command in the QDispPool. // &&&uk especially the communication parts. - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() a"); // Build the uberjob payload for each job. nlohmann::json uj; @@ -145,7 +145,7 @@ bool UberJob::runUberJob() { auto const description = jbPtr->getDescription(); LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1a"); if (description == nullptr) { - throw util::Bug(ERR_LOC, getIdStr() + " &&&uj description=null for job=" + jbPtr->getIdStr()); + throw util::Bug(ERR_LOC, cName(__func__) + " &&&uj description=null for job=" + jbPtr->getIdStr()); } auto const jsForWorker = jbPtr->getDescription()->getJsForWorker(); LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1b"); @@ -163,7 +163,7 @@ bool UberJob::runUberJob() { LOGS(_log, LOG_LVL_WARN, __func__ << " &&&REQ " << request); 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]); + cName(__func__) << " czarPost url=" << url << " request=" << request.dump() << " headers=" << headers[0]); http::Client client(method, url, request.dump(), headers); bool transmitSuccess = false; string exceptionWhat; @@ -182,15 +182,14 @@ bool UberJob::runUberJob() { } if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, "&&&uj UberJob::runUberJob() transmit failure, try to send jobs elsewhere"); - LOGS(_log, LOG_LVL_ERROR, "UberJob::" << __func__ << " transmit failure, try to send jobs elsewhere"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " transmit failure, try to send jobs elsewhere"); _unassignJobs(); // locks _jobsMtx setStatusIfOk(qmeta::JobStatus::RESPONSE_ERROR, - string("UberJob::") + __func__ + " not transmitSuccess " + exceptionWhat); + cName(__func__) + " not transmitSuccess " + exceptionWhat); } else { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() register all jobs as transmitted to worker"); - setStatusIfOk(qmeta::JobStatus::REQUEST, - string("UberJob::") + __func__ + " transmitSuccess"); // locks _jobsMtx + setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx } return false; } @@ -205,30 +204,38 @@ void UberJob::prepScrubResults() { 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; + } + auto maxAttempts = exec->getMaxAttempts(); for (auto&& job : _jobs) { - string idStr = job->getIdStr(); + string jid = job->getIdStr(); if (!job->unassignFromUberJob(getJobId())) { - auto exec = _executive.lock(); - if (exec != nullptr) { - LOGS(_log, LOG_LVL_ERROR, " max attempts for " << idStr << " reached, cancelling"); - exec->squash(); - return; - } + 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; + } + auto attempts = job->getAttemptCount(); + if (attempts > maxAttempts) { + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " job=" << jid << " attempts=" << attempts << " maxAttempts reached, cancelling"); + exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "max attempts reached " + to_string(attempts) + " job=" + jid, util::ErrorCode::INTERNAL); + exec->squash(); + return; } + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " job=" << jid << " attempts=" << attempts); } _jobs.clear(); - auto const exec = _executive.lock(); - if (exec != nullptr) { - bool const setFlag = true; - exec->setFlagFailedUberJob(setFlag); - } + bool const setFlag = true; + exec->setFlagFailedUberJob(setFlag); } bool UberJob::isQueryCancelled() { - QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); auto exec = _executive.lock(); if (exec == nullptr) { - LOGS(_log, LOG_LVL_WARN, "_executive == nullptr"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " _executive == nullptr"); return true; // Safer to assume the worst. } return exec->getCancelled(); @@ -237,7 +244,7 @@ bool UberJob::isQueryCancelled() { bool UberJob::verifyPayload() const { proto::ProtoImporter pi; if (!pi.messageAcceptable(_payload)) { - LOGS(_log, LOG_LVL_DEBUG, _idStr << " Error serializing UberJobMsg."); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " Error serializing UberJobMsg."); return false; } return true; @@ -250,7 +257,7 @@ bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg // has already done, so doing it a second time would be an error. if (newState <= currentState) { LOGS(_log, LOG_LVL_WARN, - getIdStr() << "UberJob::" << __func__ << " could not change from state=" + cName(__func__) << " could not change from state=" << _jobStatus->stateStr(currentState) << " to " << _jobStatus->stateStr(newState)); return false; } @@ -258,7 +265,7 @@ bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg // 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=" + cName(__func__) << " already error current=" << _jobStatus->stateStr(currentState) << " new=" << _jobStatus->stateStr(newState)); return false; } @@ -328,11 +335,11 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile d"); - LOGS(_log, LOG_LVL_DEBUG, __func__ << " _jobIdStr=" << getIdStr() << ", fileSize=" << fileSize); + 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, getIdStr() << " &&&uj setStatusFail could not set status to RESPONSE_READY"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " &&&uj setStatusFail could not set status to RESPONSE_READY"); return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); } @@ -345,8 +352,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc a"); auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, - "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + LOGS(_log, LOG_LVL_DEBUG, "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); return; } uint64_t resultRows = 0; @@ -380,16 +386,13 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ json UberJob::workerError(int errorCode, string const& errorMsg) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError a"); LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError code=" << errorCode << " msg=" << errorMsg); - LOGS(_log, LOG_LVL_INFO, "UberJob::workerError code=" << errorCode << " msg=" << errorMsg); - - //&&&uj NEED CODE update status for each job in this uberjob - //jq->getStatus()->updateInfo(_jobIdStr, JobStatus::RESPONSE_READY, "SSI"); + 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, "UberJob::workerError no executive or cancelled"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); return _workerErrorFinish(deleteData, "cancelled"); } LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError c"); @@ -398,7 +401,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { int dataIgnored = exec->incrDataIgnoredCount(); if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, - "UberJob ignoring, enough rows already " + cName(__func__) << " ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } return _workerErrorFinish(keepData, "none", "limitRowComplete"); @@ -406,17 +409,24 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError d"); - /* &&& - JobBase::Ptr jBaseThis = shared_from_this(); - weak_ptr ujThis = std::dynamic_pointer_cast(jBaseThis); - */ - - //&&&uj get error message to the user and kill the user query. - //&&&jq->getDescription()->respHandler()->flushHttpError(errorCode, errorMsg, util::ErrorCode::MYSQLEXEC); - int errState = util::ErrorCode::MYSQLEXEC; - getRespHandler()->flushHttpError(errorCode, errorMsg, errState); - exec->addMultiError(errorCode, errorMsg, errState); - exec->squash(); + // Currently there are no detecable 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 + // trivia to detect. A worker local database error may also qualify. + bool recoverableError = false; + recoverableError = true; //&&& delete after testing + if (recoverableError) { // &&& instead of killing the query, try to retry the jobs on a different worker + /* &&& + * + */ + _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, ""); @@ -430,11 +440,10 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str auto exec = _executive.lock(); if (exec != nullptr) { LOGS(_log, LOG_LVL_ERROR, - "UberJob::" << __func__ << " uberJobId=" << getJobId() << " shouldCancel=" << shouldCancel + cName(__func__) << " shouldCancel=" << shouldCancel << " errorType=" << errorType << " " << note); if (shouldCancel) { - LOGS(_log, LOG_LVL_ERROR, - "UberJob::" << __func__ << " uberJobId=" << getJobId() << " failing jobs"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failing jobs"); callMarkCompleteFunc(false); // all jobs failed, no retry exec->squash(); } else { @@ -443,14 +452,13 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str /// 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, - "UberJob::" << __func__ << " uberJobId=" << getJobId() << " reassigning jobs"); + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " reassigning jobs"); _unassignJobs(); exec->assignJobsToUberJobs(); } } else { LOGS(_log, LOG_LVL_INFO, - "UberJob::" << __func__ << " uberJobId=" << getJobId() << " already cancelled shouldCancel=" + cName(__func__) << " already cancelled shouldCancel=" << shouldCancel << " errorType=" << errorType << " " << note); } return jsRet; @@ -465,12 +473,12 @@ nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); if (!statusSet) { - LOGS(_log, LOG_LVL_DEBUG, "UberJob::" << __func__ << " failed to set status " << getIdStr()); + 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, "UberJob::" << __func__ << " executive is null " << getIdStr()); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; } @@ -494,8 +502,7 @@ nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& e /// Return a "success:1" json message to be sent to the worker. auto exec = _executive.lock(); if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, - "UberJob::_workerErrorFinish executive is null qId=" << getQueryId() << " ujId=" << getJobId()); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); return {{"success", 0}, {"errortype", "cancelled"}, {"note", "executive is null"}}; } diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 4a0ad126d7..6942b3b757 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -60,6 +60,10 @@ class UberJob : public JobBase { 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; } // TODO:UJ relocate to JobBase UberJobId getJobId() const override { return _uberJobId; } // &&&uj change name std::string const& getIdStr() const override { return _idStr; } @@ -145,16 +149,13 @@ class UberJob : public JobBase { qmeta::JobStatus::Ptr _jobStatus{new qmeta::JobStatus()}; // &&&uj The JobStatus class should be changed // to better represent UberJobs - //&&& std::shared_ptr _queryRequestPtr; - //&&&std::mutex _qrMtx; - std::string _payload; ///< XrdSsi message to be sent to the _workerResource. //&&&uj remove when possible std::weak_ptr _executive; std::shared_ptr _respHandler; QueryId const _queryId; UberJobId const _uberJobId; - qmeta::CzarId _czarId; + qmeta::CzarId const _czarId; std::string const _idStr; std::shared_ptr _qdispPool; //&&&uj needed? diff --git a/src/qmeta/JobStatus.h b/src/qmeta/JobStatus.h index 4e361844ac..89ecda0c84 100644 --- a/src/qmeta/JobStatus.h +++ b/src/qmeta/JobStatus.h @@ -66,8 +66,9 @@ class JobStatus { RESPONSE_DONE, CANCEL, RESPONSE_ERROR, // Errors must be between CANCEL and COMPLETE - RESULT_ERROR, // &&&uj ERRORS and CANCEL should probably be - MERGE_ERROR, // &&&uj separate from State, but may cause issues. + RESULT_ERROR, + MERGE_ERROR, + RETRY_ERROR, COMPLETE = 2000 }; From 8353ac86eea25cab06dc39eeaa941dcc2582c5ad Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 2 Jul 2024 09:45:05 -0700 Subject: [PATCH 08/15] Fixed some issues with retries. --- src/ccontrol/UserQuerySelect.cc | 276 ++------------------------------ src/czar/Czar.cc | 10 +- src/qdisp/Executive.cc | 17 +- src/qdisp/JobDescription.cc | 32 ++-- src/qdisp/JobDescription.h | 4 +- src/qdisp/JobQuery.cc | 5 +- src/qdisp/JobQuery.h | 1 + src/qdisp/UberJob.cc | 11 +- 8 files changed, 60 insertions(+), 296 deletions(-) diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index c034ff530e..bdf551ac1a 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -88,6 +88,7 @@ #include "proto/worker.pb.h" #include "proto/ProtoImporter.h" #include "qdisp/Executive.h" +#include "qdisp/JobQuery.h" #include "qmeta/MessageStore.h" #include "qmeta/QMeta.h" #include "qmeta/Exceptions.h" @@ -239,94 +240,6 @@ std::string UserQuerySelect::getResultQuery() const { return resultQuery; } -#if 0 // &&& -/// Begin running on all chunks added so far. -void UserQuerySelect::submit() { // &&& to be deleted - _qSession->finalize(); - - // Using the QuerySession, generate query specs (text, db, chunkId) and then - // create query messages and send them to the async query manager. - LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect beginning submission"); - assert(_infileMerger); - - auto taskMsgFactory = std::make_shared(); - TmpTableName ttn(_qMetaQueryId, _qSession->getOriginal()); - std::vector chunks; - std::mutex chunksMtx; - int 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 QStatsTmp table entry - try { - _queryStatsData->queryStatsTmpRegister(_qMetaQueryId, _qSession->getChunksSize()); - } catch (qmeta::SqlError const& e) { - LOGS(_log, LOG_LVL_WARN, "Failed queryStatsTmpRegister " << e.what()); - } - - _executive->setScanInteractive(_qSession->getScanInteractive()); - - for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); - ++i) { - auto& chunkSpec = *i; - - std::function funcBuildJob = [this, sequence, // sequence must be a copy - &chunkSpec, &queryTemplates, &chunks, &chunksMtx, - &ttn, &taskMsgFactory](util::CmdData*) { - QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); - - qproc::ChunkQuerySpec::Ptr cs; - { - std::lock_guard lock(chunksMtx); - bool const fillInChunkIdTag = false; - cs = _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec, fillInChunkIdTag); - chunks.push_back(cs->chunkId); - } - std::string chunkResultName = ttn.make(cs->chunkId); - - 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), taskMsgFactory, cs, - chunkResultName); - _executive->add(jobDesc); - }; - - auto cmd = std::make_shared(funcBuildJob); - _executive->queueJobStart(cmd); - ++sequence; - } - - // attempt to restore original thread priority, requires root - if (increaseThreadPriority) { - threadPriority.restoreOriginalValues(); - } - - LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); - _executive->waitForAllJobsToStart(); - - // we only care about per-chunk info for ASYNC queries - if (_async) { - std::lock_guard lock(chunksMtx); - _qMetaAddChunks(chunks); - } -} -#endif //&&& - /// Begin running on all chunks added so far. void UserQuerySelect::submit() { //&&&uj LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew start"); @@ -395,10 +308,6 @@ void UserQuerySelect::submit() { //&&&uj dbNameSet = true; } - //&&& TODO:UJ for UberJobs, cmr and MerginHandler wont be needed unless the uber job fails. could - // probably save some time. - //&&& std::shared_ptr cmr = ChunkMsgReceiver::newInstance(cs->chunkId, - //_messageStore); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d3"); ResourceUnit ru; ru.setAsDbChunk(cs->db, cs->chunkId); @@ -433,7 +342,6 @@ void UserQuerySelect::submit() { //&&&uj LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e"); //&&&if (uberJobsEnabled || true) { -#if 1 // &&& if (uberJobsEnabled) { LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); _maxChunksPerUberJob = 2; // &&&uj maybe put in config??? or set on command line?? @@ -444,162 +352,6 @@ void UserQuerySelect::submit() { //&&&uj buildAndSendUberJobs(); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e2"); -#else // &&& - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); - vector uberJobs; - - auto czarPtr = czar::Czar::getCzar(); - auto czChunkMap = czarPtr->getCzarChunkMap(); - auto czRegistry = czarPtr->getCzarRegistry(); - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1a"); - auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj - - // Make a map of all jobs in the executive. - // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can - // be found for all databases in the query - qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1b"); - - // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use - // the same databases) Use this to check for conflicts - - // assign jobs to uberJobs - int maxChunksPerUber = 3; // &&&uj maybe put in config??? or set on command line?? - // &&&uj Different queries may benefit from different values - // &&&uj Such as LIMIT=1 may work best with this at 1, where - // &&&uj 100 would be better for others. - // keep cycling through workers until no more chunks to place. - - // TODO:UJ &&&uj So UberJobIds don't conflict with chunk numbers or jobIds, start at a large number. - // This could use some refinement. - int uberJobId = qdisp::UberJob::getFirstIdNumber(); - - // &&&uj - // - create a map of UberJobs key=, val=> - // - for chunkId in `chunksInQuery` - // - 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, append a new uberjob to the vect - // - 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; - - // chunksInQuery 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. - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c"); - for (auto const& [chunkId, jqPtr] : chunksInQuery) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c1"); - auto iter = chunkMapPtr->find(chunkId); - if (iter == chunkMapPtr->end()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c2"); - missingChunks.push_back(chunkId); - break; - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c3"); - czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; - auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c4"); - if (targetWorker == nullptr) { - LOGS(_log, LOG_LVL_ERROR, "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, "Alternate worker found for chunk=" << chunkData->dump()); - } - } - if (!found) { - // &&&uj If enough workers are down, there will be a chunk that cannot be found. - // the correct course of action is probably to check the Registry, and - // after so many attempts, cancel the user query with a - // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount - // in the Job or JobDescription could be used for this. - LOGS(_log, LOG_LVL_ERROR, - "No primary or alternate worker found for chunk=" << chunkData->dump()); - throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + - " Crashing the program here for this reason is not " - "appropriate. &&& NEEDS CODE"); - } - } - // 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); - } - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c7"); - if (!missingChunks.empty()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c8"); - string errStr = string(__func__) + " a worker could not be found for these chunks "; - for (auto const& chk : missingChunks) { - errStr += to_string(chk) + ","; - } - LOGS(_log, LOG_LVL_ERROR, errStr); - throw util::Bug( - ERR_LOC, - errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); - - //&&&uj - // Add worker contact info to UberJobs. - auto const wContactMap = czRegistry->getWorkerContactMap(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f"); - for (auto const& [wIdKey, ujVect] : workerJobMap) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f1"); - auto iter = wContactMap->find(wIdKey); - if (iter == wContactMap->end()) { - // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. - throw util::Bug(ERR_LOC, string(" &&&uj NEED CODE, no contact information for ") + wIdKey); - } - auto const& wContactInfo = iter->second; - for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f2"); - ujPtr->setWorkerContactInfo(wContactInfo); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f3"); - _executive->addUberJobs(ujVect); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f4"); - for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f5"); - //&&&uj IMPORTANT - //&&&uj This just sends the test message to the worker, but the - // worker only parses it and sends a message back. The - // worker does not create and run tasks at this point. - // The call to runUberJob here should be replaced by a call - // to startUberJob that puts the call to runUberJob into - // the a priority queue command. - _executive->runUberJob(ujPtr); - } - } -#endif // &&& } LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj @@ -635,17 +387,10 @@ void UserQuerySelect::buildAndSendUberJobs() { auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj // Make a map of all jobs in the executive. - // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can - // be found for all databases in the query - /* &&& instead of destroying the only copy, going to make a copy that only contains - * unassigned jobs. - qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); - if (_chunkToJobMapInvalid.exchange(true)) { - throw util::Bug(ERR_LOC, "getChunkJobMapInvalidate called when map already invalid"); - } - return _chunkToJobMap; + // &&& TODO:UJ At some point, need to check that ResourceUnit databases can + // &&& be found for all databases in the query. + // &&& NEED CODE to use database family instead of making this check. - */ qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs c"); @@ -695,10 +440,12 @@ void UserQuerySelect::buildAndSendUberJobs() { if (iter == chunkMapPtr->end()) { LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d1a"); missingChunks.push_back(chunkId); - break; + bool const increaseAttemptCount = true; + jqPtr->getDescription()->incrAttemptCountScrubResultsJson(_executive, increaseAttemptCount); + // Assign as many jobs as possible. Any chunks not found will be attempted later. + continue; } LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d2"); - ; czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; auto targetWorker = chunkData->getPrimaryScanWorker().lock(); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d3"); @@ -762,10 +509,11 @@ void UserQuerySelect::buildAndSendUberJobs() { for (auto const& chk : missingChunks) { errStr += to_string(chk) + ","; } + errStr += " they will be retried later."; LOGS(_log, LOG_LVL_ERROR, errStr); - throw util::Bug( - ERR_LOC, - errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); + // There are likely to be unassigned jobs, so set a flag to try to make + // new uber jobs for these jobs. + _executive->setFlagFailedUberJob(true); } LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs e"); diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 01c083753a..046b9ee650 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -180,7 +180,7 @@ void Czar::_monitor() { _czarChunkMap->read(); LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor d"); - /// Create new UberJobs for all jobs that got unassigned for any reason. + /// Create new UberJobs for all jobs that are unassigned for any reason. map> execMap; { // Make a copy of all valid Executives @@ -203,6 +203,14 @@ void Czar::_monitor() { execVal->assignJobsToUberJobs(); } LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor f"); + + // TODO:UJ 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. + // Or, possibly just have the worker try to re-send the file ready message to + // the czar. } LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor end"); } diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index f6549fb828..f0ebe253ef 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -417,7 +417,9 @@ string Executive::dumpUberJobCounts() const { void Executive::assignJobsToUberJobs() { auto uqs = _userQuerySelect.lock(); - uqs->buildAndSendUberJobs(); + if (uqs != nullptr) { + uqs->buildAndSendUberJobs(); + } } bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& @@ -535,15 +537,7 @@ void Executive::markCompleted(JobId 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 != qmeta::JobStatus::CANCEL && jState != qmeta::JobStatus::RESPONSE_ERROR && - jState != qmeta::JobStatus::RESULT_ERROR && jState != qmeta::JobStatus::MERGE_ERROR) { - job->getStatus()->updateInfo(id, qmeta::JobStatus::RESULT_ERROR, "EXECFAIL", err.getCode(), - err.getMsg()); - } - */ + // Don't overwrite existing error states. job->getStatus()->updateInfoNoErrorOverwrite(id, qmeta::JobStatus::RESULT_ERROR, "EXECFAIL", err.getCode(), err.getMsg()); @@ -584,6 +578,9 @@ 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. LOGS(_log, LOG_LVL_DEBUG, "Executive::squash done"); } diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 2a31278971..03210d2566 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -36,6 +36,7 @@ #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" @@ -64,7 +65,7 @@ JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, R _chunkResultName(chunkResultName), _mock(mock) {} -bool JobDescription::incrAttemptCountScrubResults() { +bool JobDescription::incrAttemptCountScrubResults() { // &&& to be deleted if (_attemptCount >= 0) { _respHandler->prepScrubResults(_jobId, _attemptCount); // Registers the job-attempt as invalid } @@ -77,27 +78,28 @@ bool JobDescription::incrAttemptCountScrubResults() { return true; } -bool JobDescription::incrAttemptCountScrubResultsJson() { -#if 0 //&&&uj this block needs to be reenabled but attempts need to be handled differently ??? - //&&&uj attempt failures generally result from communictaion problems. SQL errors kill the query. - //&&&uj so lots of failed attempts indicate that qserv is unstable. - if (_attemptCount >= 0) { - _respHandler->prepScrubResults(_jobId, _attemptCount); // +bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase) { + + if (increase) { + ++_attemptCount; } - ++_attemptCount; - if (_attemptCount > MAX_JOB_ATTEMPTS) { + if (_attemptCount >= MAX_JOB_ATTEMPTS) { LOGS(_log, LOG_LVL_ERROR, "attemptCount greater than maximum number of retries " << _attemptCount); return false; } -#endif // &&& - ++_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 - //_payloads[_attemptCount] = os.str(); auto js = _taskMsgFactory->makeMsgJson(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, _attemptCount, _czarId); LOGS(_log, LOG_LVL_ERROR, "&&& JobDescription::incrAttemptCountScrubResultsJson js=" << (*js)); diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index a6ec53cbf7..131bc56712 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -55,6 +55,7 @@ class TaskMsgFactory; namespace qdisp { +class Executive; class ResponseHandler; /** Description of a job managed by the executive @@ -90,7 +91,8 @@ class JobDescription { /// If the starting value of _attemptCount was greater than or equal to zero, that /// attempt is scrubbed from the result table. bool incrAttemptCountScrubResults(); // &&&uj - to be deleted - bool incrAttemptCountScrubResultsJson(); // &&&uj - scrubbing results probably unneeded with uj. + /// doc &&&uj - scrubbing results probably unneeded with uj. This should be renamed. + bool incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase); bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 9a0db86835..0ba2b32586 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -200,7 +200,10 @@ bool JobQuery::unassignFromUberJob(UberJobId ujId) { return false; } _uberJobId = -1; - _jobDescription->incrAttemptCountScrubResultsJson(); + + 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; } diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 76e5fd4e08..3cc1d9ee21 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -182,6 +182,7 @@ class JobQuery : public JobBase { /// 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; }; diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 0b0586f8f0..ad02b9d04e 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -106,9 +106,10 @@ bool UberJob::runUberJob() { LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&&uj count qid=" << getQueryId() << " ujId=" << getJobId() << " jobs.sz=" << _jobs.size()); + auto exec = _executive.lock(); for (auto const& jqPtr : _jobs) { - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() a1"); - jqPtr->getDescription()->incrAttemptCountScrubResultsJson(); + LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() a1 " << jqPtr->getIdStr()); + jqPtr->getDescription()->incrAttemptCountScrubResultsJson(exec, true); } LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() b"); @@ -209,7 +210,7 @@ void UberJob::_unassignJobs() { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " exec is null"); return; } - auto maxAttempts = exec->getMaxAttempts(); + //&&&auto maxAttempts = exec->getMaxAttempts(); for (auto&& job : _jobs) { string jid = job->getIdStr(); if (!job->unassignFromUberJob(getJobId())) { @@ -218,6 +219,7 @@ void UberJob::_unassignJobs() { exec->squash(); return; } + /* &&& auto attempts = job->getAttemptCount(); if (attempts > maxAttempts) { LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " job=" << jid << " attempts=" << attempts << " maxAttempts reached, cancelling"); @@ -225,7 +227,8 @@ void UberJob::_unassignJobs() { exec->squash(); return; } - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " job=" << jid << " attempts=" << attempts); + */ + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); } _jobs.clear(); bool const setFlag = true; From bd5dd53c944e154155540f92fae07c649c3812d8 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 9 Jul 2024 17:11:48 -0700 Subject: [PATCH 09/15] Integration tests past. --- src/admin/python/lsst/qserv/admin/itest.py | 1 + src/ccontrol/MergingHandler.cc | 34 --- src/ccontrol/UserQuerySelect.cc | 39 ++- src/ccontrol/UserQuerySelect.h | 5 +- src/czar/Czar.cc | 20 +- src/czar/Czar.h | 7 +- src/czar/CzarChunkMap.cc | 302 ++++++++++++++++++++- src/czar/CzarChunkMap.h | 132 ++++++++- src/czar/testCzar.cc | 19 ++ src/global/intTypes.h | 4 +- src/qdisp/Executive.cc | 10 + src/qdisp/Executive.h | 32 +-- src/qdisp/JobDescription.cc | 2 + src/qdisp/JobDescription.h | 2 +- src/qdisp/UberJob.cc | 32 ++- src/qmeta/QMetaMysql.cc | 8 +- src/rproc/InfileMerger.h | 2 +- src/wbase/Task.cc | 62 +++-- src/wbase/UberJobData.cc | 4 +- src/wbase/UberJobData.h | 4 +- src/wdb/QueryRunner.cc | 1 + src/xrdsvc/HttpWorkerCzarModule.cc | 81 +++--- 22 files changed, 619 insertions(+), 184 deletions(-) diff --git a/src/admin/python/lsst/qserv/admin/itest.py b/src/admin/python/lsst/qserv/admin/itest.py index 9235cfd65f..73794bc253 100644 --- a/src/admin/python/lsst/qserv/admin/itest.py +++ b/src/admin/python/lsst/qserv/admin/itest.py @@ -969,6 +969,7 @@ def compareQueryResults(run_cases: List[str], outputs_dir: str) -> List[ITestCas 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/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 77eba7b84c..7d6f44be66 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -736,42 +736,11 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe LOGS(_log, LOG_LVL_ERROR, __func__ << " failed, jobBase was NULL"); return {success, shouldCancel}; // both should still be false } - //&&& auto const jobQuery = std::dynamic_pointer_cast(jobBase); auto const uberJob = std::dynamic_pointer_cast(jobBase); LOGS(_log, LOG_LVL_TRACE, "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); - /* &&& errors will be handled by MergingHandler::flushHttpError() - if (responseSummary.errorcode() != 0 || !responseSummary.errormsg().empty()) { - _error = util::Error(responseSummary.errorcode(), responseSummary.errormsg(), - util::ErrorCode::MYSQLEXEC); - _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); - LOGS(_log, LOG_LVL_ERROR, - "MergingHandler::" << __func__ << " error from worker:" << responseSummary.wname() - << " error: " << _error); - return false; - } - */ - - /* &&&& - // 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 size, bool& last) { - last = true; - proto::ResponseData responseData; - if (responseData.ParseFromArray(buf, size) && responseData.IsInitialized()) { - bool const success = _merge(responseSummary, responseData, jobQuery); - if (success) { - resultRows += responseData.row_size(); - last = resultRows >= responseSummary.rowcount(); - } - return success; - } - throw runtime_error("MergingHandler::flush ** message deserialization failed **"); - }; - */ // 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'. @@ -781,11 +750,8 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe proto::ResponseData responseData; if (responseData.ParseFromArray(buf, bufSize) && responseData.IsInitialized()) { bool const mergeSuccess = _mergeHttp(uberJob, responseData); - //&&&if (responseData.IsInitialized()) { - //&&&bool const mergeSuccess = _mergeHttp(uberJob, responseData); if (mergeSuccess) { resultRows += responseData.row_size(); - //&&&last = resultRows >= responseSummary.rowcount(); last = resultRows >= expectedRows; } return mergeSuccess; diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index bdf551ac1a..a90885dcf3 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -276,8 +276,8 @@ void UserQuerySelect::submit() { //&&&uj _executive->setScanInteractive(_qSession->getScanInteractive()); - string dbName(""); // it isn't easy to set this //&&&diff - bool dbNameSet = false; //&&&diff + string dbName(""); + bool dbNameSet = false; LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d"); for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); @@ -288,7 +288,7 @@ void UserQuerySelect::submit() { //&&&uj // Make the JobQuery now QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); - qproc::ChunkQuerySpec::Ptr cs; //&&&diff old one did this in lambda + qproc::ChunkQuerySpec::Ptr cs; { std::lock_guard lock(chunksMtx); cs = _qSession->buildChunkQuerySpec(queryTemplates, chunkSpec); @@ -298,11 +298,10 @@ void UserQuerySelect::submit() { //&&&uj LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d2"); // This should only need to be set once as all jobs should have the same database name. - //&&& this probably has to do with locating xrootd resources, need to check. ??? if (cs->db != dbName) { LOGS(_log, LOG_LVL_WARN, "&&& dbName change from " << dbName << " to " << cs->db); if (dbNameSet) { - throw util::Bug(ERR_LOC, "Multiple database names in UBerJob"); + throw util::Bug(ERR_LOC, "Multiple database names in UberJob"); } dbName = cs->db; dbNameSet = true; @@ -313,7 +312,6 @@ void UserQuerySelect::submit() { //&&&uj ru.setAsDbChunk(cs->db, cs->chunkId); qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( _qMetaCzarId, _executive->getId(), sequence, ru, - //&&&std::make_shared(cmr, _infileMerger, chunkResultName), std::make_shared(_infileMerger, chunkResultName), taskMsgFactory, cs, chunkResultName); auto job = _executive->add(jobDesc); @@ -337,6 +335,10 @@ void UserQuerySelect::submit() { //&&&uj ++sequence; } + if (dbNameSet) { + _queryDbName = dbName; + } + /// &&& ******************************************************** /// &&&uj at this point the executive has a map of all jobs with the chunkIds as the key. @@ -349,6 +351,7 @@ void UserQuerySelect::submit() { //&&&uj // &&&uj Such as LIMIT=1 may work best with this at 1, where // &&&uj 100 would be better for others. //&&&_executive->buildAndSendUberJobs(maxChunksPerUber); + _executive->setReadyToExecute(); buildAndSendUberJobs(); LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e2"); @@ -356,7 +359,7 @@ void UserQuerySelect::submit() { //&&&uj LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); - _executive->waitForAllJobsToStart(); + _executive->waitForAllJobsToStart(); // &&& this may not be needed anymore? // we only care about per-chunk info for ASYNC queries if (_async) { @@ -371,6 +374,14 @@ void UserQuerySelect::buildAndSendUberJobs() { LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs a"); string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); LOGS(_log, LOG_LVL_INFO, funcN << " start"); + + // Ensure `_monitor()` doesn't do anything until everything is ready. + if (!_executive->isReadyToExecute()) { + LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " 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); @@ -380,11 +391,23 @@ void UserQuerySelect::buildAndSendUberJobs() { vector uberJobs; auto czarPtr = czar::Czar::getCzar(); - auto czChunkMap = czarPtr->getCzarChunkMap(); + //&&&auto czChunkMap = czarPtr->getCzarChunkMap(); + 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(); + // TODO:UJ It may be better to just fail the query now, but with a working + // system, this should be very rare. + } + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs b"); auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj + LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs b1"); // Make a map of all jobs in the executive. // &&& TODO:UJ At some point, need to check that ResourceUnit databases can diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index a69e07b1be..b346d92030 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -200,7 +200,10 @@ class UserQuerySelect : public UserQuery { std::atomic _uberJobIdSeq{900'000}; ///< &&&uj can probably start at 1 std::shared_ptr _ttn; ///< Temporary table name generator. - /// &&&uj Only one thread should run buildAndSendUberJobs() at a time + /// Primary database name for the query. + std::string _queryDbName; + + /// &&&uj Only one thread should run buildAndSendUberJobs() at a time. std::mutex _buildUberJobMtx; }; diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 046b9ee650..fa34fb031d 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -177,7 +177,13 @@ void Czar::_monitor() { /// Check database for changes in worker chunk assignments and aliveness - _czarChunkMap->read(); + //&&&_czarChunkMap->read(); + _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`. LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor d"); /// Create new UberJobs for all jobs that are unassigned for any reason. @@ -204,13 +210,16 @@ void Czar::_monitor() { } LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor f"); - // TODO:UJ get missing results from workers. + // 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. - // Or, possibly just have the worker try to re-send the file ready message to - // the czar. + // 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? } LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor end"); } @@ -239,7 +248,8 @@ Czar::Czar(string const& configFilePath, string const& czarName) _czarConfig->setId(_uqFactory->userQuerySharedResources()->qMetaCzarId); try { - _czarChunkMap = CzarChunkMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + //&&& _czarChunkMap = CzarChunkMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); + _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); } catch (ChunkMapException const& exc) { LOGS(_log, LOG_LVL_WARN, string(__func__) + " failed to create CzarChunkMap " + exc.what()); } diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 9e110d9ff9..dd6fea3fb8 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -66,7 +66,7 @@ class Executive; namespace lsst::qserv::czar { -class CzarChunkMap; +class CzarFamilyMap; class CzarRegistry; /// @addtogroup czar @@ -134,7 +134,7 @@ class Czar { /// @return The reconstructed info for the query SubmitResult getQueryInfo(QueryId queryId) const; - std::shared_ptr getCzarChunkMap() const { return _czarChunkMap; } + std::shared_ptr getCzarFamilyMap() const { return _czarFamilyMap; } std::shared_ptr getCzarRegistry() const { return _czarRegistry; } @@ -203,7 +203,8 @@ class Czar { std::shared_ptr _controlHttpSvc; /// Map of which chunks on which workers and shared scan order. - std::shared_ptr _czarChunkMap; + // &&& std::shared_ptr _czarChunkMap; + std::shared_ptr _czarFamilyMap; /// Connection to the registry to register the czar and get worker contact information. std::shared_ptr _czarRegistry; diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 5c36e466d3..bc2cec67a1 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -45,9 +45,10 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarChunkMap"); namespace lsst::qserv::czar { +/* &&& CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { try { - auto mapsSet = _read(); + auto mapsSet = _readOld(); if (!mapsSet) { throw ChunkMapException(ERR_LOC, "CzarChunkMap maps were not set in contructor"); } @@ -56,20 +57,25 @@ CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta( throw ChunkMapException(ERR_LOC, string(" CzarChunkMap constructor failed read ") + qExc.what()); } } +*/ + +CzarChunkMap::CzarChunkMap() { +} CzarChunkMap::~CzarChunkMap() { LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); } -bool CzarChunkMap::read() { +/* &&& +bool CzarChunkMap::readOld() { bool mapsSet = false; try { - mapsSet = _read(); + mapsSet = _readOld(); } catch (qmeta::QMetaError const& qExc) { LOGS(_log, LOG_LVL_ERROR, __func__ << " CzarChunkMap could not read DB " << qExc.what()); } return mapsSet; } -bool CzarChunkMap::_read() { +bool CzarChunkMap::_readOld() { LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_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. @@ -84,22 +90,23 @@ bool CzarChunkMap::_read() { } // Make the new maps. - auto [chunkMapPtr, wcMapPtr] = makeNewMaps(qChunkMap); + auto [chunkMapPtr, wcMapPtr] = makeNewMapsOld(qChunkMap); - verify(*chunkMapPtr, *wcMapPtr); + verifyOld(*chunkMapPtr, *wcMapPtr); LOGS(_log, LOG_LVL_DEBUG, " chunkMap=" << dumpChunkMap(*chunkMapPtr)); LOGS(_log, LOG_LVL_DEBUG, " workerChunkMap=" << dumpWorkerChunkMap(*wcMapPtr)); _workerChunkMap = wcMapPtr; _chunkMap = chunkMapPtr; - _lastUpdateTime = qChunkMap.updateTime; + //&&&_lastUpdateTime = qChunkMap.updateTime; LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_read() end"); return true; } +*/ -pair, shared_ptr> CzarChunkMap::makeNewMaps( +pair, shared_ptr> CzarChunkMap::makeNewMapsOld( qmeta::QMetaChunkMap const& qChunkMap) { // Create new maps. auto wcMapPtr = make_shared(); @@ -228,7 +235,7 @@ void CzarChunkMap::insertIntoChunkMap(WorkerChunkMap& wcMap, ChunkMap& chunkMap, chunkData->addToWorkerHasThis(workerChunksData); } -void CzarChunkMap::calcChunkMap(ChunkMap& chunkMap, ChunkVector& chunksSortedBySize) { +void CzarChunkMap::calcChunkMap(ChunkMap const& chunkMap, ChunkVector& chunksSortedBySize) { // Calculate total bytes for all chunks. for (auto&& [chunkIdNum, chunkData] : chunkMap) { chunkData->_calcTotalBytes(); @@ -249,7 +256,9 @@ void CzarChunkMap::sortChunks(std::vector& chunksSortedBySize) { std::sort(chunksSortedBySize.begin(), chunksSortedBySize.end(), sortBySizeDesc); } -void CzarChunkMap::verify(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap) { +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; @@ -300,10 +309,15 @@ void CzarChunkMap::verify(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap) } 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{"; @@ -347,6 +361,49 @@ CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() const { return newMap; } +void CzarChunkMap::organize() { + + auto chunksSortedBySize = make_shared(); + + //&&&calcChunkMap(*chunkMapPtr, *chunksSortedBySize); + 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(); @@ -379,4 +436,229 @@ string CzarChunkMap::WorkerChunksData::dump() const { 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) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& a"); + // Create new maps. + std::shared_ptr newFamilyMap = make_shared(); + //&&&auto wcMapPtr = make_shared(); + //&&&auto chunkMapPtr = make_shared(); + + // Workers -> Databases map + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& b workers.sz=" << qChunkMap.workers.size()); + for (auto const& [workerId, dbs] : qChunkMap.workers) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& c " << workerId << " dbs.sz=" << dbs.size()); + // Databases -> Tables map + for (auto const& [dbName, tables] : dbs) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& d " << dbName << " tbls.sz=" << tables.size() ); + // Tables -> Chunks map + for (auto const& [tableName, chunks] : tables) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& e " << tableName << " chunks.sz=" << chunks.size()); + // vector of ChunkInfo + for (qmeta::QMetaChunkMap::ChunkInfo const& chunkInfo : chunks) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& f"); + try { + int64_t chunkNum = chunkInfo.chunk; + CzarChunkMap::SizeT sz = chunkInfo.size; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& g"); + LOGS(_log, LOG_LVL_DEBUG, + "workerdId=" << workerId << " db=" << dbName << " table=" << tableName + << " chunk=" << chunkNum << " sz=" << sz); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& h"); + insertIntoMaps(newFamilyMap, workerId, dbName, tableName, chunkNum, sz); + } catch (invalid_argument const& exc) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& h1"); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& EXCEPTION a"); + throw ChunkMapException( + ERR_LOC, string(__func__) + " invalid_argument workerdId=" + workerId + + " db=" + dbName + " table=" + tableName + + " chunk=" + to_string(chunkInfo.chunk) + " " + exc.what()); + } catch (out_of_range const& exc) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& h2"); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& EXCEPTION b"); + throw ChunkMapException( + ERR_LOC, string(__func__) + " out_of_range workerdId=" + workerId + + " db=" + dbName + " table=" + tableName + + " chunk=" + to_string(chunkInfo.chunk) + " " + exc.what()); + } + } + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& e " << tableName << " end"); + } + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& d " << dbName << " end"); + } + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& c " << workerId << " end"); + } + + // this needs to be done for each CzarChunkMap in the family map. + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& i"); + for(auto&& [familyName, chunkMapPtr] : *newFamilyMap) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& j"); + LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap::makeNewMaps working on " << familyName); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& k"); + + chunkMapPtr->organize(); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& l"); + } + + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& end"); + 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) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& a"); + // Get the CzarChunkMap for this family + auto familyName = getFamilyNameFromDbName(dbName); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& b"); + 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; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c"); + auto familyIter = nfMap.find(familyName); + if (familyIter == nfMap.end()) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c1"); + czarChunkMap = CzarChunkMap::Ptr(new CzarChunkMap()); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c1a"); + nfMap[familyName] = czarChunkMap; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c1b"); + } else { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c2"); + czarChunkMap = familyIter->second; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c2a"); + } + + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& d"); + auto [chunkMapPtr, wcMapPtr] = czarChunkMap->_getMaps(); + + // &&& This bit of indirection is both no longer needed and confusing. + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& e"); + CzarChunkMap::WorkerChunkMap& wcMap = *wcMapPtr; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& f"); + CzarChunkMap::ChunkMap& chunkMap = *chunkMapPtr; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& g"); + + // Get or make the worker entry + CzarChunkMap::WorkerChunksData::Ptr workerChunksData; + auto iterWC = wcMap.find(workerId); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h"); + if (iterWC == wcMap.end()) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h1"); + workerChunksData = CzarChunkMap::WorkerChunksData::Ptr(new CzarChunkMap::WorkerChunksData(workerId)); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h1a"); + wcMap[workerId] = workerChunksData; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h1b"); + } else { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h2"); + workerChunksData = iterWC->second; + } + + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& i"); + // Get or make the ChunkData entry in chunkMap + CzarChunkMap::ChunkData::Ptr chunkData; + auto iterChunkData = chunkMap.find(chunkIdNum); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j"); + if (iterChunkData == chunkMap.end()) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j1"); + chunkData = CzarChunkMap::ChunkData::Ptr(new CzarChunkMap::ChunkData(chunkIdNum)); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j1a"); + chunkMap[chunkIdNum] = chunkData; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j1b"); + } else { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j2"); + chunkData = iterChunkData->second; + } + + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& k"); + // Set or verify the table information + auto iterDT = chunkData->_dbTableMap.find({dbName, tableName}); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l"); + if (iterDT == chunkData->_dbTableMap.end()) { + // doesn't exist so set it up + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l1"); + chunkData->_dbTableMap[{dbName, tableName}] = sz; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l1a"); + } else { + // Verify that it matches other data + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l2"); + auto const& dbTbl = iterDT->first; + auto tblSz = iterDT->second; + auto const& dbN = dbTbl.first; + auto const& tblN = dbTbl.second; + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l2a"); + if (dbName != dbN || tblN != tableName || tblSz != sz) { + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l2a1"); + LOGS(_log, LOG_LVL_ERROR, + __func__ << " data mismatch for " << dbName << "." << tableName << "=" << sz << " vs " << dbN + << "." << tblN << "=" << tblSz); + } + } + + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& m"); + // Link WorkerData the single chunkData instance for the chunkId + workerChunksData->_chunkDataMap[chunkIdNum] = chunkData; + + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& n"); + // Add worker to the list of workers containing the chunk. + chunkData->addToWorkerHasThis(workerChunksData); + LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& end"); +} + +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 index c3e5fa432c..54057e836d 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -43,6 +43,8 @@ struct QMetaChunkMap; namespace lsst::qserv::czar { +class CzarFamilyMap; + class ChunkMapException : public util::Issue { public: ChunkMapException(Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} @@ -85,11 +87,12 @@ class CzarChunkMap { using Ptr = std::shared_ptr; using SizeT = uint64_t; - CzarChunkMap() = delete; + //&&&CzarChunkMap() = delete; 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(std::shared_ptr const& qmeta) { return Ptr(new CzarChunkMap(qmeta)); } + static Ptr create() { return Ptr(new CzarChunkMap()); } ~CzarChunkMap(); @@ -118,6 +121,7 @@ class CzarChunkMap { std::string dump() const; friend CzarChunkMap; + friend CzarFamilyMap; private: int64_t const _chunkId; ///< The Id number for this chunk. @@ -154,6 +158,7 @@ class CzarChunkMap { std::string dump() const; friend CzarChunkMap; + friend CzarFamilyMap; private: std::string const _workerId; @@ -191,20 +196,23 @@ class CzarChunkMap { /// @param `sz` - size in bytes of the table being inserted. static void insertIntoChunkMap(WorkerChunkMap& wcMap, ChunkMap& chunkMap, std::string const& workerId, std::string const& dbName, std::string const& tableName, - int64_t chunkIdNum, SizeT sz); + int64_t chunkIdNum, SizeT sz); /// &&& delete /// Calculate the total bytes in each chunk and then sort the resulting ChunkVector by chunk size, /// descending. - static void calcChunkMap(ChunkMap& chunkMap, ChunkVector& chunksSortedBySize); + static void calcChunkMap(ChunkMap const& chunkMap, ChunkVector& chunksSortedBySize); /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. static std::pair, std::shared_ptr> - makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); + makeNewMapsOld(qmeta::QMetaChunkMap const& qChunkMap); //&&& delete /// Verify that all chunks belong to at least one worker and that all chunks are represented in shared /// scans. /// @throws ChunkMapException - static void verify(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap); + static void verifyOld(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap); // &&& delete + void verify(); + + //void setMaps(std::shared_ptr const& chunkMapPtr, std::shared_ptr const& wcMapPtr); // &&& delete static std::string dumpChunkMap(ChunkMap const& chunkMap); @@ -220,31 +228,131 @@ class CzarChunkMap { } /// &&& doc - bool read(); + void organize(); + + /// &&& doc + //&&&bool readOld(); private: + /* &&& /// Try to `_read` values for maps from `qmeta`. CzarChunkMap(std::shared_ptr const& qmeta); + */ + 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}; + } + /// Read the json worker list from the database and update the maps if there's a new /// version since the `_lastUpdateTime`. /// @throws `qmeta::QMetaError` - bool _read(); + //&&& bool _readOld(); - std::shared_ptr _qmeta; ///< Database connection to collect json worker list. + //&&&std::shared_ptr _qmeta; ///< Database connection to collect json worker list. /// Map of all workers and which chunks they contain. - std::shared_ptr _workerChunkMap; + 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; + std::shared_ptr _chunkMap{new ChunkMap()}; + + /* &&& + /// The last time the maps were updated with information from the replicator. + TIMEPOINT _lastUpdateTime; // initialized to 0; + */ + mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap (&&& still needed???) + + friend CzarFamilyMap; +}; + +/// &&& doc +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); + } + + + /// &&& doc + bool read(); + + /// &&& doc + /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. + //&&&static std::pair, std::shared_ptr> + std::shared_ptr makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); + + // &&& doc + 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); + + // &&& + static void verify(std::shared_ptr const& familyMap); + +private: + + /// Try to `_read` values for maps from `qmeta`. + CzarFamilyMap(std::shared_ptr const& qmeta); + + /// &&& doc + 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; - mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap, _timeStamp, and _qmeta. + std::shared_ptr _familyMap{new FamilyMapType()}; + mutable std::mutex _familyMapMtx; ///< protects _familyMap, _timeStamp, and _qmeta. }; } // namespace lsst::qserv::czar diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index ace8fc86a3..9ab805282f 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -186,6 +186,7 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } )"; + /* &&& auto jsTest1 = nlohmann::json::parse(test1); qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); @@ -197,6 +198,24 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { tie(chunkMapPtr, wcMapPtr) = czar::CzarChunkMap::makeNewMaps(qChunkMap2); czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 2 passed"); + */ + + auto dbToFamily = make_shared(); + czar::CzarFamilyMap czFamMap(dbToFamily); + + + auto jsTest1 = nlohmann::json::parse(test1); + qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); + //&&&auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); + 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/intTypes.h b/src/global/intTypes.h index 1693940539..c3a6f7fb07 100644 --- a/src/global/intTypes.h +++ b/src/global/intTypes.h @@ -47,8 +47,8 @@ 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. diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index f0ebe253ef..05dc89b875 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -316,6 +316,7 @@ void Executive::runUberJob(std::shared_ptr const& uberJob) { }; auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); + _jobStartCmdList.push_back(cmd); if (_scanInteractive) { _qdispPool->queCmd(cmd, 0); } else { @@ -581,6 +582,8 @@ void Executive::squash() { // 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. + bool const deleteResults = true; + sendWorkerCancelMsg(deleteResults); LOGS(_log, LOG_LVL_DEBUG, "Executive::squash done"); } @@ -608,9 +611,16 @@ 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) { + LOGS(_log, LOG_LVL_ERROR, "&&& NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId + queryId."); +} + int Executive::getNumInflight() const { unique_lock lock(_incompleteJobsMutex); return _incompleteJobs.size(); diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 2a6c560468..73c4ca43df 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -239,6 +239,16 @@ class Executive : public std::enable_shared_from_this { 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, SharedResources::Ptr const& sharedResources, std::shared_ptr const& qStatus, @@ -302,21 +312,6 @@ class Executive : public std::enable_shared_from_this { QueryId _id = 0; ///< Unique identifier for this query. std::string _idStr{QueryIdHelper::makeIdStr(0, true)}; - /* &&& - qmeta::CzarId _qMetaCzarId; ///< Czar ID in QMeta database - - /// temporary table name generator, which uses a hash and jobId or uberJobId - /// to generate names for tables. This cannot be set until after `_id` is - /// set. - std::shared_ptr _ttn; - - /// Pointer to the result merging class. - std::shared_ptr _infileMerger; - - /// UberJobIds for need to be unique within each UserQuery. - std::atomic _uberJobId = qdisp::UberJob::getFirstIdNumber(); - */ - std::shared_ptr _qMeta; /// Last time Executive updated QMeta, defaults to epoch for clock. std::chrono::system_clock::time_point _lastQMetaUpdate; @@ -365,10 +360,9 @@ class Executive : public std::enable_shared_from_this { /// executive can make new uberjobs. std::atomic _failedUberJob{false}; - /* &&& - static std::mutex _executiveMapMtx; ///< protects _executiveMap - static std::map> _executiveMap; ///< Map of executives for queries in progress. - */ + /// Flag that is set to true when ready to create and run UberJobs. + std::atomic _readyToExecute{false}; + }; /// &&&uj MarkCompleteFunc is not needed with uberjobs. diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index 03210d2566..fd3b64c16a 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -115,12 +115,14 @@ void JobDescription::buildPayload() { _payloads[_attemptCount] = os.str(); } +/* &&& bool JobDescription::fillTaskMsg(proto::TaskMsg* tMsg) { //&&&uj -probably just delete. //&&&uj return _taskMsgFactory->fillTaskMsg(tMsg, *_chunkQuerySpec, _chunkResultName, _queryId, //_jobId, _attemptCount, _czarId); util::Bug(ERR_LOC, "&&& JobDescription::fillTaskMsg"); return false; } +*/ bool JobDescription::verifyPayload() const { //&&&uj - is there any value to this now? proto::ProtoImporter pi; diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 131bc56712..106e4b9edf 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -95,7 +95,7 @@ class JobDescription { bool incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase); bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. - bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj + //&&&bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj std::shared_ptr getJsForWorker() { return _jsForWorker; } diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index ad02b9d04e..3a8bcc896f 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -310,8 +310,10 @@ void UberJob::callMarkCompleteFunc(bool success) { json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_t fileSize) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile a"); LOGS(_log, LOG_LVL_WARN, - "&&&uj UberJob::importResultFile fileUrl=" << fileUrl << " rowCount=" << rowCount + cName(__func__) << "&&&uj fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileUrl=" << fileUrl + << " rowCount=" << rowCount << " fileSize=" << fileSize); if (isQueryCancelled()) { LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile import job was cancelled."); @@ -321,7 +323,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ auto exec = _executive.lock(); if (exec == nullptr || exec->getCancelled()) { - LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile no executive or cancelled"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive or cancelled"); return _importResultError(true, "cancelled", "Query cancelled - no executive"); } LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile c"); @@ -361,7 +363,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ uint64_t resultRows = 0; auto [flushSuccess, flushShouldCancel] = ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc b"); + LOGS(_log, LOG_LVL_WARN, ujPtr->cName(__func__) << "::fileCollectFunc &&&uj b"); if (!flushSuccess) { // This would probably indicate malformed file+rowCount or // writing the result table failed. @@ -412,24 +414,25 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError d"); - // Currently there are no detecable 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 - // trivia to detect. A worker local database error may also qualify. + // 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; //&&& delete after testing - if (recoverableError) { // &&& instead of killing the query, try to retry the jobs on a different worker - /* &&& - * - */ + recoverableError = true; //&&& 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 {// &&& + } 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, ""); @@ -469,6 +472,7 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish a"); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << "&&&uj start"); // &&& keep /// 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 diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index c8095acff9..8b06966b43 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -883,6 +883,7 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c 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_WARN, "&&& QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table << " chunk=" << chunk << " sz=" << size); } chunkMap.updateTime = updateTime; } catch (exception const& ex) { @@ -896,7 +897,7 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock sql::SqlErrorObject errObj; sql::SqlResults results; string const tableName = "chunkMapStatus"; - string const query = "SELECT `update_time` FROM `" + tableName + "` ORDER BY `update_time` DESC LIMIT 1"; + 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); @@ -913,6 +914,11 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock throw ConsistencyError(ERR_LOC, "Too many rows in result set of query " + query); } try { + int j=0; // &&& del + for (auto const& str : updateTime) { // &&& del + LOGS(_log, LOG_LVL_WARN, "&&& _updatetime j=" << j << " Insrt=" << str << " stol=" << stol(str)); + ++j; + } 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/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index 5053207634..cf9e5d0581 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -283,7 +283,7 @@ class InfileMerger { std::mutex _queryIdStrMtx; ///< protects _queryIdStr std::atomic _queryIdStrSet{false}; - std::string _queryIdStr{"QI=?"}; ///< Unknown until results start coming back from workers. + std::string _queryIdStr{"QID=?"}; ///< Unknown until results start coming back from workers. std::string _jobIdColName; ///< Name of the jobId column in the result table. int const _jobIdMysqlType{MYSQL_TYPE_LONG}; ///< 4 byte integer. diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index 3dd1b97af8..ac933e0763 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -407,86 +407,89 @@ std::vector Task::createTasksForChunk( 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); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jobdesc " << jsJobDesc); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC jobdesc " << jsJobDesc); // See qproc::TaskMsgFactory::makeMsgJson for message construction. - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k1"); auto const jdCzarId = rbJobDesc.required("czarId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); + //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k2"); 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)); } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); + //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k3"); auto const jdJobId = rbJobDesc.required("jobId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); + //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k4"); auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k5"); + //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k5"); auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); + //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k6"); auto const jdScanPriority = rbJobDesc.required("scanPriority"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); auto const jdChunkId = rbJobDesc.required("chunkId"); LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + funcN << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); auto const jdQueryFragments = rbJobDesc.required("queryFragments"); int fragmentNumber = 0; //&&&uj should this be 1??? Is this at all useful? for (auto const& frag : jdQueryFragments) { vector fragSubQueries; vector fragSubchunkIds; vector fragSubTables; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC frag=" << frag); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC frag=" << frag); http::RequestBody rbFrag(frag); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); auto const& jsQueries = rbFrag.required("queries"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); // &&&uj move to uberjob???, these should be the same for all jobs for (auto const& subQ : jsQueries) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c1"); http::RequestBody rbSubQ(subQ); auto const subQuery = rbSubQ.required("subQuery"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC subQuery=" << subQuery); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC subQuery=" << subQuery); fragSubQueries.push_back(subQuery); } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d1"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k10d1"); auto const& resultTable = rbFrag.required("resultTable"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); auto const& jsSubIds = rbFrag.required("subchunkIds"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId jsSubIds=" << jsSubIds); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scId jsSubIds=" << jsSubIds); for (auto const& scId : jsSubIds) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scId=" << scId); + //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scId=" << scId); fragSubchunkIds.push_back(scId); } auto const& jsSubTables = rbFrag.required("subchunkTables"); for (auto const& scDbTable : jsSubTables) { // &&&uj are these the same for all jobs? http::RequestBody rbScDbTable(scDbTable); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10f1"); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k10f1"); string scDb = rbScDbTable.required("scDb"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scDb=" << scDb); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scDb=" << scDb); string scTable = rbScDbTable.required("scTable"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC scTable=" << scDbTable); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scTable=" << scDbTable); TaskDbTbl scDbTbl(scDb, scTable); fragSubTables.push_back(scDbTbl); } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC fragSubQueries.sz=" << fragSubQueries.size()); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC fragSubQueries.sz=" << fragSubQueries.size()); for (string const& fragSubQ : fragSubQueries) { size_t templateId = userQueryInfo->addTemplate(fragSubQ); if (fragSubchunkIds.empty()) { @@ -498,6 +501,7 @@ std::vector Task::createTasksForChunk( fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); // &&& change to make_shared vect.push_back(task); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC fragSubchunkIds.empty()==true vect.sz=" << vect.size() << " fragNum=" << fragmentNumber); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; @@ -507,6 +511,7 @@ std::vector Task::createTasksForChunk( scanInteractive, maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); // &&& change to make_shared vect.push_back(task); + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC fragSubchunkIds.empty()==false vect.sz=" << vect.size() << " fragNum=" << fragmentNumber); } } } @@ -514,7 +519,10 @@ std::vector Task::createTasksForChunk( } } + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC vect.sz=" << vect.size()); + for (auto taskPtr : vect) { + LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC taskPtr calling setTaskQueryRunner"); // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, sqlConnMgr, queriesAndChunks)); diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index f4ade93149..9fa6a0b036 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -76,7 +76,7 @@ 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 = getFuncIdStr(__func__); + string const funcN = cName(__func__); LOGS(_log, LOG_LVL_WARN, funcN << "&&& UberJobData::responseFileReady a httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize << " headerCount=" << headerCount); @@ -138,7 +138,7 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled) { LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseError a"); - string const funcN = getFuncIdStr(__func__); + string const funcN = cName(__func__); string errorMsg; int errorCode = 0; diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index 565928f40d..c12ba2ff9c 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -92,8 +92,8 @@ class UberJobData { bool responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled); std::string getIdStr() const { return _idStr; } - std::string getFuncIdStr(std::string const& funcName) { - return getIdStr() + " UberJobData::" + funcName + " "; + std::string cName(std::string const& funcName) { + return "UberJobData::" + funcName + " " + getIdStr(); } private: diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index a4a7557ab5..be3224d8eb 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -218,6 +218,7 @@ bool QueryRunner::runQuery() { MYSQL_RES* QueryRunner::_primeResult(string const& query) { util::HoldTrack::Mark mark(ERR_LOC, "QR _primeResult() QID=" + _task->getIdStr()); + LOGS(_log, LOG_LVL_WARN, "&&& " << _task->getIdStr() << " QueryRunner::_primeResult " << query); bool queryOk = _mysqlConn->queryUnbuffered(query); if (!queryOk) { sql::SqlErrorObject errObj; diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 3c633cf7af..dcc95bee89 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -158,6 +158,12 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { wbase::FileChannelShared::create(ujData, czarId, czarHostName, czarPort, targetWorkerId); ujData->setFileChannelShared(channelShared); + QueryId jdQueryId = 0; + proto::ScanInfo scanInfo; // &&& + bool scanInfoSet = false; + bool jdScanInteractive = false; + int jdMaxTableSize = 0; + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); for (auto const& job : ujJobs) { json const& jsJobDesc = job["jobdesc"]; @@ -167,7 +173,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); auto const jdCzarId = rbJobDesc.required("czarId"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); - auto const jdQueryId = rbJobDesc.required("queryId"); + jdQueryId = rbJobDesc.required("queryId"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); auto const jdJobId = rbJobDesc.required("jobId"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); @@ -177,9 +183,9 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); auto const jdScanPriority = rbJobDesc.required("scanPriority"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); - auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); + jdScanInteractive = rbJobDesc.required("scanInteractive"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); - auto const jdMaxTableSize = rbJobDesc.required("maxTableSize"); + jdMaxTableSize = rbJobDesc.required("maxTableSize"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); auto const jdChunkId = rbJobDesc.required("chunkId"); LOGS(_log, LOG_LVL_WARN, @@ -191,37 +197,44 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { //&&&uj need scan table info befor making tasks LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11"); //&&&proto::ScanTableInfo::ListOf scanTables; - proto::ScanInfo scanInfo; + //&&&proto::ScanInfo scanInfo; auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); - for (auto const& tbl : jdChunkScanTables) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a1"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC tbl=" << tbl); - http::RequestBody rbTbl(tbl); - auto const& chunkScanDb = rbTbl.required("db"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a2"); - auto const& lockInMemory = rbTbl.required("lockInMemory"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a3"); - auto const& chunkScanTable = rbTbl.required("table"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a4"); - auto const& tblScanRating = rbTbl.required("tblScanRating"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory - << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); - scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, tblScanRating); + if (!scanInfoSet) { + for (auto const& tbl : jdChunkScanTables) { + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a1"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC tbl=" << tbl); + http::RequestBody rbTbl(tbl); + auto const& chunkScanDb = rbTbl.required("db"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a2"); + auto const& lockInMemory = rbTbl.required("lockInMemory"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a3"); + auto const& chunkScanTable = rbTbl.required("table"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a4"); + auto const& tblScanRating = rbTbl.required("tblScanRating"); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory + << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); + scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, tblScanRating); + scanInfoSet = true; + } } LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k12"); scanInfo.scanRating = jdScanPriority; LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); - // 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()); } + + // create tasks and add them to ujData + auto chunkTasks = wbase::Task::createTasksForChunk( // &&& getting called twice when it should only be called once + ujData, ujJobs, channelShared, scanInfo, jdScanInteractive, jdMaxTableSize, + foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), foreman()->sqlConnMgr(), + foreman()->queriesAndChunks(), foreman()->httpPort()); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC chunkTasks.sz=" << chunkTasks.size() << " QID=" << jdQueryId); + ujTasks.insert(ujTasks.end(), chunkTasks.begin(), chunkTasks.end()); + LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC ujTasks.sz=" << ujTasks.size() << " QID=" << jdQueryId); + channelShared->setTaskCount(ujTasks.size()); ujData->addTasks(ujTasks); @@ -232,22 +245,6 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&& Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); -#if 0 /// &&&&&&&& - // 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. //&&&uj next - t.stop(); - LOGS(_log, LOG_LVL_DEBUG, - "Enqueued TaskMsg for " << ru << " in " << t.getElapsed() << " seconds"); - break; - } -#endif /// &&&&&&&& - // &&&uj temporary, send response back to czar saying file is ready. The file is not ready, but this // is just an initial comms test //&&&_temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); From 2f8a467817f59adf1a4756c26d49a13a7b41d394 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 9 Jul 2024 17:23:19 -0700 Subject: [PATCH 10/15] Reformatted. --- src/ccontrol/MergingHandler.cc | 2 +- src/ccontrol/MergingHandler.h | 2 +- src/ccontrol/UserQuerySelect.cc | 5 +-- src/czar/Czar.cc | 6 ++-- src/czar/Czar.h | 12 +++---- src/czar/CzarChunkMap.cc | 40 +++++++++++----------- src/czar/CzarChunkMap.h | 29 ++++++++-------- src/czar/testCzar.cc | 1 - src/qdisp/Executive.cc | 13 ++++--- src/qdisp/Executive.h | 9 ++--- src/qdisp/JobDescription.cc | 11 +++--- src/qdisp/JobDescription.h | 4 +-- src/qdisp/JobQuery.cc | 2 +- src/qdisp/UberJob.cc | 55 ++++++++++++++++-------------- src/qdisp/UberJob.h | 4 +-- src/qmeta/QMetaMysql.cc | 11 +++--- src/wbase/Task.cc | 48 ++++++++++++++------------ src/wbase/UberJobData.h | 4 +-- src/xrdsvc/HttpWorkerCzarModule.cc | 16 +++++---- 19 files changed, 140 insertions(+), 134 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 7d6f44be66..58c79ba349 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -779,7 +779,7 @@ void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, _flushError(jq); */ - if(!_errorSet.exchange(true)) { + if (!_errorSet.exchange(true)) { _error = util::Error(errorCode, errorMsg, util::ErrorCode::MYSQLEXEC); _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); } diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index 2a228c88f7..edc5f5a68b 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -130,7 +130,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}; ///< &&& doc + std::atomic _errorSet{false}; ///< &&& doc 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/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index a90885dcf3..2300055e48 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -359,7 +359,7 @@ void UserQuerySelect::submit() { //&&&uj LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); - _executive->waitForAllJobsToStart(); // &&& this may not be needed anymore? + _executive->waitForAllJobsToStart(); // &&& this may not be needed anymore? // we only care about per-chunk info for ASYNC queries if (_async) { @@ -377,7 +377,8 @@ void UserQuerySelect::buildAndSendUberJobs() { // Ensure `_monitor()` doesn't do anything until everything is ready. if (!_executive->isReadyToExecute()) { - LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " executive isn't ready to generate UberJobs."); + LOGS(_log, LOG_LVL_DEBUG, + "UserQuerySelect::" << __func__ << " executive isn't ready to generate UberJobs."); return; } diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index fa34fb031d..73a686a099 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -170,12 +170,11 @@ Czar::Ptr Czar::createCzar(string const& configFilePath, string const& czarName) void Czar::_monitor() { LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor a"); - while(_monitorLoop) { + while (_monitorLoop) { LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor b"); this_thread::sleep_for(_monitorSleepTime); LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor c"); - /// Check database for changes in worker chunk assignments and aliveness //&&&_czarChunkMap->read(); _czarFamilyMap->read(); @@ -192,7 +191,7 @@ void Czar::_monitor() { // Make a copy of all valid Executives lock_guard execMapLock(_executiveMapMtx); auto iter = _executiveMap.begin(); - while(iter != _executiveMap.end()) { + while (iter != _executiveMap.end()) { auto qIdKey = iter->first; shared_ptr exec = iter->second.lock(); if (exec == nullptr) { @@ -325,7 +324,6 @@ Czar::Czar(string const& configFilePath, string const& czarName) // Start the monitor thread thread monitorThrd(&Czar::_monitor, this); _monitorThrd = move(monitorThrd); - } Czar::~Czar() { diff --git a/src/czar/Czar.h b/src/czar/Czar.h index dd6fea3fb8..74eacdb5f4 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -63,7 +63,6 @@ namespace lsst::qserv::qdisp { class Executive; } // namespace lsst::qserv::qdisp - namespace lsst::qserv::czar { class CzarFamilyMap; @@ -209,14 +208,13 @@ class Czar { /// 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::mutex _executiveMapMtx; ///< protects _executiveMap - std::map> _executiveMap; ///< Map of executives for queries in progress. - - std::thread _monitorThrd; ///< &&& doc - std::atomic _monitorLoop{true}; ///< &&& doc + std::thread _monitorThrd; ///< &&& doc + std::atomic _monitorLoop{true}; ///< &&& doc std::chrono::milliseconds _monitorSleepTime{15000}; ///< Wait time between checks. &&& set from config - }; } // namespace lsst::qserv::czar diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index bc2cec67a1..59c0e28713 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -59,8 +59,7 @@ CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta( } */ -CzarChunkMap::CzarChunkMap() { -} +CzarChunkMap::CzarChunkMap() {} CzarChunkMap::~CzarChunkMap() { LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); } @@ -106,8 +105,8 @@ bool CzarChunkMap::_readOld() { } */ -pair, shared_ptr> CzarChunkMap::makeNewMapsOld( - qmeta::QMetaChunkMap const& qChunkMap) { +pair, shared_ptr> +CzarChunkMap::makeNewMapsOld(qmeta::QMetaChunkMap const& qChunkMap) { // Create new maps. auto wcMapPtr = make_shared(); auto chunkMapPtr = make_shared(); @@ -314,10 +313,8 @@ void CzarChunkMap::verify() { // 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{"; @@ -362,7 +359,6 @@ CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() const { } void CzarChunkMap::organize() { - auto chunksSortedBySize = make_shared(); //&&&calcChunkMap(*chunkMapPtr, *chunksSortedBySize); @@ -385,8 +381,8 @@ void CzarChunkMap::organize() { continue; // maybe the next one will be okay. } LOGS(_log, LOG_LVL_DEBUG, - __func__ << " wkrId=" << wkrData << " tsz=" << wkrData->_sharedScanTotalSize - << " smallest=" << smallest); + __func__ << " wkrId=" << wkrData << " tsz=" << wkrData->_sharedScanTotalSize + << " smallest=" << smallest); if (wkrData->_sharedScanTotalSize < smallest) { smallestWkr = wkrData; smallest = smallestWkr->_sharedScanTotalSize; @@ -394,13 +390,13 @@ void CzarChunkMap::organize() { } if (smallestWkr == nullptr) { throw ChunkMapException(ERR_LOC, string(__func__) + " no smallesWkr found for chunk=" + - to_string(chunkData->_chunkId)); + 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); + " chunk=" << chunkData->_chunkId << " assigned to scan on " << smallestWkr->_workerId); } } @@ -436,8 +432,6 @@ string CzarChunkMap::WorkerChunksData::dump() const { return os.str(); } - - CzarFamilyMap::CzarFamilyMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { try { auto mapsSet = _read(); @@ -469,8 +463,8 @@ bool CzarFamilyMap::_read() { 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)); + << util::TimeUtils::timePointToDateTimeString(_lastUpdateTime) + << " db=" << util::TimeUtils::timePointToDateTimeString(qChunkMap.updateTime)); return false; } @@ -501,10 +495,12 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& c " << workerId << " dbs.sz=" << dbs.size()); // Databases -> Tables map for (auto const& [dbName, tables] : dbs) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& d " << dbName << " tbls.sz=" << tables.size() ); + LOGS(_log, LOG_LVL_WARN, + "CzarFamilyMap::makeNewMaps &&& d " << dbName << " tbls.sz=" << tables.size()); // Tables -> Chunks map for (auto const& [tableName, chunks] : tables) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& e " << tableName << " chunks.sz=" << chunks.size()); + LOGS(_log, LOG_LVL_WARN, + "CzarFamilyMap::makeNewMaps &&& e " << tableName << " chunks.sz=" << chunks.size()); // vector of ChunkInfo for (qmeta::QMetaChunkMap::ChunkInfo const& chunkInfo : chunks) { LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& f"); @@ -542,7 +538,7 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( // this needs to be done for each CzarChunkMap in the family map. LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& i"); - for(auto&& [familyName, chunkMapPtr] : *newFamilyMap) { + for (auto&& [familyName, chunkMapPtr] : *newFamilyMap) { LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& j"); LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap::makeNewMaps working on " << familyName); LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& k"); @@ -556,13 +552,15 @@ std::shared_ptr CzarFamilyMap::makeNewMaps( } void CzarFamilyMap::insertIntoMaps(std::shared_ptr const& newFamilyMap, string const& workerId, - string const& dbName, string const& tableName, int64_t chunkIdNum, - CzarChunkMap::SizeT sz) { + string const& dbName, string const& tableName, int64_t chunkIdNum, + CzarChunkMap::SizeT sz) { LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& a"); // Get the CzarChunkMap for this family auto familyName = getFamilyNameFromDbName(dbName); LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& b"); - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " familyInsrt{w=" << workerId << " fN=" << familyName << " dbN=" << dbName << " tblN=" << tableName << " chunk=" << chunkIdNum << " sz=" << sz << "}"); + 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; LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c"); diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index 54057e836d..de9f8a6095 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -91,7 +91,7 @@ class CzarChunkMap { 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(std::shared_ptr const& qmeta) { return Ptr(new CzarChunkMap(qmeta)); } static Ptr create() { return Ptr(new CzarChunkMap()); } ~CzarChunkMap(); @@ -196,7 +196,7 @@ class CzarChunkMap { /// @param `sz` - size in bytes of the table being inserted. static void insertIntoChunkMap(WorkerChunkMap& wcMap, ChunkMap& chunkMap, std::string const& workerId, std::string const& dbName, std::string const& tableName, - int64_t chunkIdNum, SizeT sz); /// &&& delete + int64_t chunkIdNum, SizeT sz); /// &&& delete /// Calculate the total bytes in each chunk and then sort the resulting ChunkVector by chunk size, /// descending. @@ -204,15 +204,16 @@ class CzarChunkMap { /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. static std::pair, std::shared_ptr> - makeNewMapsOld(qmeta::QMetaChunkMap const& qChunkMap); //&&& delete + makeNewMapsOld(qmeta::QMetaChunkMap const& qChunkMap); //&&& delete /// Verify that all chunks belong to at least one worker and that all chunks are represented in shared /// scans. /// @throws ChunkMapException - static void verifyOld(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap); // &&& delete + static void verifyOld(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap); // &&& delete void verify(); - //void setMaps(std::shared_ptr const& chunkMapPtr, std::shared_ptr const& wcMapPtr); // &&& delete + // void setMaps(std::shared_ptr const& chunkMapPtr, std::shared_ptr const& + // wcMapPtr); // &&& delete static std::string dumpChunkMap(ChunkMap const& chunkMap); @@ -242,14 +243,12 @@ class CzarChunkMap { /// Return shared pointers to `_chunkMap` and `_workerChunkMap`, which should be held until /// finished with the data. - std::pair, - std::shared_ptr> + std::pair, std::shared_ptr> _getMaps() const { std::lock_guard lck(_mapMtx); return {_chunkMap, _workerChunkMap}; } - /// Read the json worker list from the database and update the maps if there's a new /// version since the `_lastUpdateTime`. /// @throws `qmeta::QMetaError` @@ -269,7 +268,7 @@ class CzarChunkMap { /// The last time the maps were updated with information from the replicator. TIMEPOINT _lastUpdateTime; // initialized to 0; */ - mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap (&&& still needed???) + mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap (&&& still needed???) friend CzarFamilyMap; }; @@ -312,30 +311,30 @@ class CzarFamilyMap { return _getChunkMap(familyName); } - /// &&& doc bool read(); /// &&& doc /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. - //&&&static std::pair, std::shared_ptr> + //&&&static std::pair, + //std::shared_ptr> std::shared_ptr makeNewMaps(qmeta::QMetaChunkMap const& qChunkMap); // &&& doc - 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); + 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); // &&& static void verify(std::shared_ptr const& familyMap); private: - /// Try to `_read` values for maps from `qmeta`. CzarFamilyMap(std::shared_ptr const& qmeta); /// &&& doc bool _read(); - /// Return the chunk map for the `familyName` CzarChunkMap::Ptr _getChunkMap(std::string const& familyName) const { std::lock_guard familyLock(_familyMapMtx); @@ -352,7 +351,7 @@ class CzarFamilyMap { TIMEPOINT _lastUpdateTime; // initialized to 0; std::shared_ptr _familyMap{new FamilyMapType()}; - mutable std::mutex _familyMapMtx; ///< protects _familyMap, _timeStamp, and _qmeta. + mutable std::mutex _familyMapMtx; ///< protects _familyMap, _timeStamp, and _qmeta. }; } // namespace lsst::qserv::czar diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index 9ab805282f..96ee576d4d 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -203,7 +203,6 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { auto dbToFamily = make_shared(); czar::CzarFamilyMap czFamMap(dbToFamily); - auto jsTest1 = nlohmann::json::parse(test1); qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); //&&&auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 05dc89b875..9d005ac30c 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -102,7 +102,8 @@ namespace lsst::qserv::qdisp { /* &&& mutex Executive::_executiveMapMtx; ///< protects _executiveMap -map> Executive::_executiveMap; ///< Map of executives for queries in progress. +map> Executive::_executiveMap; ///< Map of executives for queries in +progress. */ //////////////////////////////////////////////////////////////////////// @@ -304,11 +305,10 @@ void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect end"); } - void Executive::runUberJob(std::shared_ptr const& uberJob) { LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); - bool const useqdisppool = true; /// &&& delete + bool const useqdisppool = true; /// &&& delete if (useqdisppool) { auto runUberJobFunc = [uberJob](util::CmdData*) { LOGS(_log, LOG_LVL_WARN, "&&&uj Executive::runUberJob::runUberJobFunc a"); @@ -454,7 +454,8 @@ void Executive::addMultiError(int errorCode, std::string const& errorMsg, int er { lock_guard lock(_errorsMutex); _multiError.push_back(err); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) + " multiError:" << _multiError.size() << ":" << _multiError); + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) + " multiError:" << _multiError.size() << ":" << _multiError); } } @@ -618,7 +619,9 @@ void Executive::_squashSuperfluous() { } void Executive::sendWorkerCancelMsg(bool deleteResults) { - LOGS(_log, LOG_LVL_ERROR, "&&& NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId + queryId."); + LOGS(_log, LOG_LVL_ERROR, + "&&& NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId + " + "queryId."); } int Executive::getNumInflight() const { diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 73c4ca43df..3930d7df80 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -122,9 +122,7 @@ class Executive : public std::enable_shared_from_this { static Ptr getExecutiveFromMap(QueryId qId); */ - std::string cName(const char* funcName="") { - return std::string("Executive::") + funcName; - } + std::string cName(const char* funcName = "") { return std::string("Executive::") + funcName; } /// &&&uj doc void setUserQuerySelect(std::shared_ptr const& uqs) { _userQuerySelect = uqs; } @@ -174,7 +172,7 @@ class Executive : public std::enable_shared_from_this { //&&&void setTmpTableNameGenerator(std::shared_ptr const& ttn) { _ttn = ttn; } //&&&void setInfileMerger(std::shared_ptr infileMerger) { _infileMerger = - //infileMerger; } + // infileMerger; } QueryId getId() const { return _id; } std::string const& getIdStr() const { return _idStr; } @@ -237,7 +235,7 @@ 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; } // Should be set by config /// Calling this indicates the executive is ready to create and execute UberJobs. void setReadyToExecute() { _readyToExecute = true; } @@ -362,7 +360,6 @@ class Executive : public std::enable_shared_from_this { /// Flag that is set to true when ready to create and run UberJobs. std::atomic _readyToExecute{false}; - }; /// &&&uj MarkCompleteFunc is not needed with uberjobs. diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index fd3b64c16a..de8db60445 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -65,7 +65,7 @@ JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, R _chunkResultName(chunkResultName), _mock(mock) {} -bool JobDescription::incrAttemptCountScrubResults() { // &&& to be deleted +bool JobDescription::incrAttemptCountScrubResults() { // &&& to be deleted if (_attemptCount >= 0) { _respHandler->prepScrubResults(_jobId, _attemptCount); // Registers the job-attempt as invalid } @@ -79,7 +79,6 @@ bool JobDescription::incrAttemptCountScrubResults() { // &&& to be deleted } bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase) { - if (increase) { ++_attemptCount; } @@ -92,8 +91,12 @@ bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr 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); + 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; } diff --git a/src/qdisp/JobDescription.h b/src/qdisp/JobDescription.h index 106e4b9edf..88aa48c3b2 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -90,10 +90,10 @@ class JobDescription { /// @returns true when _attemptCount is incremented correctly and the payload is built. /// If the starting value of _attemptCount was greater than or equal to zero, that /// attempt is scrubbed from the result table. - bool incrAttemptCountScrubResults(); // &&&uj - to be deleted + bool incrAttemptCountScrubResults(); // &&&uj - to be deleted /// doc &&&uj - scrubbing results probably unneeded with uj. This should be renamed. bool incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase); - bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. + bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. //&&&bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj diff --git a/src/qdisp/JobQuery.cc b/src/qdisp/JobQuery.cc index 0ba2b32586..5ebc7c6d26 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -60,7 +60,7 @@ JobQuery::JobQuery(Executive::Ptr const& executive, JobDescription::Ptr const& j JobQuery::~JobQuery() { LOGS(_log, LOG_LVL_DEBUG, "~JobQuery"); - LOGS(_log, LOG_LVL_WARN, "~JobQuery QID=" <<_idStr); + LOGS(_log, LOG_LVL_WARN, "~JobQuery QID=" << _idStr); } /** Attempt to run the job on a worker. diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 3a8bcc896f..5aae13d330 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -146,7 +146,8 @@ bool UberJob::runUberJob() { auto const description = jbPtr->getDescription(); LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1a"); if (description == nullptr) { - throw util::Bug(ERR_LOC, cName(__func__) + " &&&uj description=null for job=" + jbPtr->getIdStr()); + throw util::Bug(ERR_LOC, + cName(__func__) + " &&&uj description=null for job=" + jbPtr->getIdStr()); } auto const jsForWorker = jbPtr->getDescription()->getJsForWorker(); LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1b"); @@ -164,7 +165,8 @@ bool UberJob::runUberJob() { LOGS(_log, LOG_LVL_WARN, __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]); + cName(__func__) << " czarPost url=" << url << " request=" << request.dump() + << " headers=" << headers[0]); http::Client client(method, url, request.dump(), headers); bool transmitSuccess = false; string exceptionWhat; @@ -186,7 +188,7 @@ bool UberJob::runUberJob() { 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); + cName(__func__) + " not transmitSuccess " + exceptionWhat); } else { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() register all jobs as transmitted to worker"); @@ -215,20 +217,21 @@ void UberJob::_unassignJobs() { 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->addMultiError(qmeta::JobStatus::RETRY_ERROR, "unable to re-assign " + jid, + util::ErrorCode::INTERNAL); exec->squash(); return; } /* &&& auto attempts = job->getAttemptCount(); if (attempts > maxAttempts) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " job=" << jid << " attempts=" << attempts << " maxAttempts reached, cancelling"); - exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "max attempts reached " + to_string(attempts) + " job=" + jid, util::ErrorCode::INTERNAL); - exec->squash(); - return; + LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " job=" << jid << " attempts=" << attempts << " + maxAttempts reached, cancelling"); exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "max attempts + reached " + to_string(attempts) + " job=" + jid, util::ErrorCode::INTERNAL); exec->squash(); return; } */ - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); } _jobs.clear(); bool const setFlag = true; @@ -260,16 +263,16 @@ bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg // 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)); + 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)); + cName(__func__) << " already error current=" << _jobStatus->stateStr(currentState) + << " new=" << _jobStatus->stateStr(newState)); return false; } @@ -311,9 +314,9 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile a"); LOGS(_log, LOG_LVL_WARN, cName(__func__) << "&&&uj fileUrl=" << fileUrl << " rowCount=" << rowCount - << " fileSize=" << fileSize); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileUrl=" << fileUrl - << " rowCount=" << rowCount << " fileSize=" << fileSize); + << " fileSize=" << fileSize); + LOGS(_log, LOG_LVL_DEBUG, + cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); if (isQueryCancelled()) { LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile import job was cancelled."); @@ -344,7 +347,8 @@ 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__) << " &&&uj setStatusFail could not set status to RESPONSE_READY"); + LOGS(_log, LOG_LVL_WARN, + cName(__func__) << " &&&uj setStatusFail could not set status to RESPONSE_READY"); return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); } @@ -357,7 +361,8 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc a"); auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); + LOGS(_log, LOG_LVL_DEBUG, + "UberJob::importResultFile::fileCollectFunction uberjob ptr is null " << fileUrl); return; } uint64_t resultRows = 0; @@ -407,7 +412,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { if ((dataIgnored - 1) % 1000 == 0) { LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ignoring, enough rows already " - << "dataIgnored=" << dataIgnored); + << "dataIgnored=" << dataIgnored); } return _workerErrorFinish(keepData, "none", "limitRowComplete"); } @@ -421,7 +426,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; //&&& delete after testing &&&&&&& + recoverableError = true; //&&& 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) @@ -446,8 +451,8 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str auto exec = _executive.lock(); if (exec != nullptr) { LOGS(_log, LOG_LVL_ERROR, - cName(__func__) << " shouldCancel=" << shouldCancel - << " errorType=" << errorType << " " << note); + 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 @@ -464,15 +469,15 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str } } else { LOGS(_log, LOG_LVL_INFO, - cName(__func__) << " already cancelled shouldCancel=" - << shouldCancel << " errorType=" << errorType << " " << note); + cName(__func__) << " already cancelled shouldCancel=" << shouldCancel + << " errorType=" << errorType << " " << note); } return jsRet; } nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish a"); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << "&&&uj start"); // &&& keep + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << "&&&uj start"); // &&& keep /// 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 diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 6942b3b757..ad25e645a1 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -60,9 +60,7 @@ class UberJob : public JobBase { bool addJob(std::shared_ptr const& job); bool runUberJob(); - std::string cName(const char* funcN) const { - return std::string("UberJob::") + funcN + " " + getIdStr(); - } + std::string cName(const char* funcN) const { return std::string("UberJob::") + funcN + " " + getIdStr(); } QueryId getQueryId() const override { return _queryId; } // TODO:UJ relocate to JobBase UberJobId getJobId() const override { return _uberJobId; } // &&&uj change name diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index 8b06966b43..8fff95cd5f 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -883,7 +883,9 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c 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_WARN, "&&& QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table << " chunk=" << chunk << " sz=" << size); + LOGS(_log, LOG_LVL_WARN, + "&&& QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table + << " chunk=" << chunk << " sz=" << size); } chunkMap.updateTime = updateTime; } catch (exception const& ex) { @@ -897,7 +899,8 @@ 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 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); @@ -914,8 +917,8 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock throw ConsistencyError(ERR_LOC, "Too many rows in result set of query " + query); } try { - int j=0; // &&& del - for (auto const& str : updateTime) { // &&& del + int j = 0; // &&& del + for (auto const& str : updateTime) { // &&& del LOGS(_log, LOG_LVL_WARN, "&&& _updatetime j=" << j << " Insrt=" << str << " stol=" << stol(str)); ++j; } diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index ac933e0763..a9a46ad2f7 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -418,61 +418,61 @@ std::vector Task::createTasksForChunk( // See qproc::TaskMsgFactory::makeMsgJson for message construction. LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k1"); auto const jdCzarId = rbJobDesc.required("czarId"); - //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k2"); + // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k2"); 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)); } - //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k3"); + // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k3"); auto const jdJobId = rbJobDesc.required("jobId"); - //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k4"); + // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k4"); auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k5"); + // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k5"); auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - //LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k6"); + // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k6"); auto const jdScanPriority = rbJobDesc.required("scanPriority"); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); auto const jdChunkId = rbJobDesc.required("chunkId"); LOGS(_log, LOG_LVL_WARN, - funcN << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId - << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb - << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive - << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); + funcN << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + << " jdAtt=" << jdAttemptCount << " jdQDb=" << jdQuerySpecDb + << " jdScanPri=" << jdScanPriority << " interactive=" << jdScanInteractive + << " maxTblSz=" << jdMaxTableSizeMb << " chunkId=" << jdChunkId); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); auto const jdQueryFragments = rbJobDesc.required("queryFragments"); int fragmentNumber = 0; //&&&uj should this be 1??? Is this at all useful? for (auto const& frag : jdQueryFragments) { vector fragSubQueries; vector fragSubchunkIds; vector fragSubTables; - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC frag=" << frag); http::RequestBody rbFrag(frag); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); auto const& jsQueries = rbFrag.required("queries"); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); - // &&&uj move to uberjob???, these should be the same for all jobs + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); + // &&&uj move to uberjob???, these should be the same for all jobs for (auto const& subQ : jsQueries) { LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c1"); http::RequestBody rbSubQ(subQ); auto const subQuery = rbSubQ.required("subQuery"); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC subQuery=" << subQuery); fragSubQueries.push_back(subQuery); } LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k10d1"); auto const& resultTable = rbFrag.required("resultTable"); - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); auto const& jsSubIds = rbFrag.required("subchunkIds"); LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scId jsSubIds=" << jsSubIds); for (auto const& scId : jsSubIds) { - //LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); + // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scId=" << scId); fragSubchunkIds.push_back(scId); } @@ -501,7 +501,9 @@ std::vector Task::createTasksForChunk( fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); // &&& change to make_shared vect.push_back(task); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC fragSubchunkIds.empty()==true vect.sz=" << vect.size() << " fragNum=" << fragmentNumber); + LOGS(_log, LOG_LVL_WARN, + funcN << "&&&SUBC fragSubchunkIds.empty()==true vect.sz=" << vect.size() + << " fragNum=" << fragmentNumber); } else { for (auto subchunkId : fragSubchunkIds) { bool const hasSubchunks = true; @@ -511,7 +513,9 @@ std::vector Task::createTasksForChunk( scanInteractive, maxTableSizeMb, fragSubTables, fragSubchunkIds, sendChannel, resultsHttpPort)); // &&& change to make_shared vect.push_back(task); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC fragSubchunkIds.empty()==false vect.sz=" << vect.size() << " fragNum=" << fragmentNumber); + LOGS(_log, LOG_LVL_WARN, + funcN << "&&&SUBC fragSubchunkIds.empty()==false vect.sz=" << vect.size() + << " fragNum=" << fragmentNumber); } } } diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index c12ba2ff9c..8316eadee6 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -92,9 +92,7 @@ class UberJobData { 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(); - } + 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, diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index dcc95bee89..4514dfb7fe 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -159,7 +159,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { ujData->setFileChannelShared(channelShared); QueryId jdQueryId = 0; - proto::ScanInfo scanInfo; // &&& + proto::ScanInfo scanInfo; // &&& bool scanInfoSet = false; bool jdScanInteractive = false; int jdMaxTableSize = 0; @@ -213,9 +213,10 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { auto const& tblScanRating = rbTbl.required("tblScanRating"); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory - << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); - scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, tblScanRating); + __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory + << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); + scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, + tblScanRating); scanInfoSet = true; } } @@ -223,15 +224,16 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { scanInfo.scanRating = jdScanPriority; LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); - } // create tasks and add them to ujData - auto chunkTasks = wbase::Task::createTasksForChunk( // &&& getting called twice when it should only be called once + auto chunkTasks = wbase::Task::createTasksForChunk( // &&& getting called twice when it should only + // be called once ujData, ujJobs, channelShared, scanInfo, jdScanInteractive, jdMaxTableSize, foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), foreman()->sqlConnMgr(), foreman()->queriesAndChunks(), foreman()->httpPort()); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC chunkTasks.sz=" << chunkTasks.size() << " QID=" << jdQueryId); + LOGS(_log, LOG_LVL_WARN, + __func__ << "&&&SUBC chunkTasks.sz=" << chunkTasks.size() << " QID=" << jdQueryId); ujTasks.insert(ujTasks.end(), chunkTasks.begin(), chunkTasks.end()); LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC ujTasks.sz=" << ujTasks.size() << " QID=" << jdQueryId); From 17171385db56220aa7084304f77bfd51de4f9cfc Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 10 Jul 2024 13:52:06 -0700 Subject: [PATCH 11/15] Code cleanup. --- src/ccontrol/MergingHandler.cc | 67 ++----- src/ccontrol/MergingHandler.h | 13 +- src/ccontrol/UserQueryFactory.cc | 3 - src/ccontrol/UserQuerySelect.cc | 145 ++++---------- src/ccontrol/UserQuerySelect.h | 26 ++- src/czar/Czar.cc | 22 +-- src/czar/Czar.h | 16 +- src/czar/CzarChunkMap.cc | 260 +----------------------- src/czar/CzarChunkMap.h | 99 +++------- src/czar/HttpCzarWorkerModule.cc | 42 +--- src/czar/HttpCzarWorkerModule.h | 9 +- src/czar/testCzar.cc | 15 -- src/http/RequestBodyJSON.h | 6 +- src/proto/worker.proto | 13 +- src/qdisp/Executive.cc | 305 ++--------------------------- src/qdisp/Executive.h | 79 ++++---- src/qdisp/JobDescription.cc | 15 +- src/qdisp/JobDescription.h | 9 +- src/qdisp/JobQuery.cc | 2 +- src/qdisp/JobQuery.h | 17 +- src/qdisp/QueryRequest.cc | 16 +- src/qdisp/QueryRequest.h | 16 +- src/qdisp/ResponseHandler.h | 4 +- src/qdisp/UberJob.cc | 113 +++-------- src/qdisp/UberJob.h | 63 +++--- src/qhttp/Server.cc | 3 - src/qmeta/JobStatus.cc | 5 - src/qmeta/QMetaMysql.cc | 11 +- src/qproc/TaskMsgFactory.cc | 66 ++----- src/qproc/TaskMsgFactory.h | 9 +- src/rproc/InfileMerger.cc | 11 +- src/rproc/InfileMerger.h | 3 +- src/wbase/FileChannelShared.cc | 73 +------ src/wbase/FileChannelShared.h | 38 +--- src/wbase/Task.cc | 115 ++--------- src/wbase/Task.h | 22 +-- src/wbase/UberJobData.cc | 43 ++-- src/wbase/UberJobData.h | 25 +-- src/wbase/UserQueryInfo.h | 2 +- src/wdb/QueryRunner.cc | 1 - src/wsched/ChunkTasksQueue.cc | 39 ---- src/wsched/ScanScheduler.cc | 1 - src/xrdsvc/HttpReplicaMgtModule.cc | 1 - src/xrdsvc/HttpSvc.cc | 2 +- src/xrdsvc/HttpWorkerCzarModule.cc | 133 ++----------- src/xrdsvc/HttpWorkerCzarModule.h | 15 +- 46 files changed, 386 insertions(+), 1607 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 58c79ba349..db79771f44 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -380,7 +380,6 @@ std::tuple readHttpFileAndMergeHttp( lsst::qserv::qdisp::UberJob::Ptr const& uberJob, string const& httpUrl, function const& messageIsReady, shared_ptr const& httpConnPool) { - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp a"); string const context = "MergingHandler::" + string(__func__) + " " + " qid=" + uberJob->getIdStr() + " "; LOGS(_log, LOG_LVL_DEBUG, context << "httpUrl=" << httpUrl); @@ -411,9 +410,8 @@ std::tuple readHttpFileAndMergeHttp( uint32_t msgSizeBytes = 0; bool success = true; bool mergeSuccess = true; - LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp b"); - int headerCount = 0; // &&& del - uint64_t totalBytesRead = 0; /// &&& del + int headerCount = 0; + uint64_t totalBytesRead = 0; try { string const noClientData; vector const noClientHeaders; @@ -423,27 +421,19 @@ std::tuple readHttpFileAndMergeHttp( clientConfig.tcpKeepAlive = true; clientConfig.tcpKeepIdle = 5; // the default is 60 sec clientConfig.tcpKeepIntvl = 5; // the default is 60 sec - LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp c"); http::Client reader(http::Method::GET, httpUrl, noClientData, noClientHeaders, clientConfig, httpConnPool); - LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp d"); reader.read([&](char const* inBuf, size_t inBufSize) { - // LOGS(_log, LOG_LVL_WARN, context + "&&& readHttpFileAndMergeHttp d1 reader.read - // ok"); // 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_WARN, - context + "&&& readHttpFileAndMergeHttp e next=" << (uint64_t)next << " end=" - << (uint64_t)end << " last=" << last); while ((next < end) && !last) { LOGS(_log, LOG_LVL_WARN, - context + "&&& readHttpFileAndMergeHttp e1 next=" << (uint64_t)next << " end=" - << (uint64_t)end << " last=" << last); + context << "TODO:UJ next=" << (uint64_t)next << " end=" << (uint64_t)end + << " last=" << last); if (msgSizeBytes == 0) { - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e2"); // Continue or finish reading the frame header. size_t const bytes2read = std::min(sizeof(uint32_t) - msgSizeBufNext, (size_t)(end - next)); @@ -453,8 +443,6 @@ std::tuple readHttpFileAndMergeHttp( msgSizeBufNext += bytes2read; if (msgSizeBufNext == sizeof(uint32_t)) { ++headerCount; - LOGS(_log, LOG_LVL_WARN, - context + "&&& readHttpFileAndMergeHttp e3 &&& headerCount=" << headerCount); // Done reading the frame header. msgSizeBufNext = 0; // Parse and evaluate the message length. @@ -482,7 +470,6 @@ std::tuple readHttpFileAndMergeHttp( } } else { // Continue or finish reading the message body. - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e4"); size_t const bytes2read = std::min((size_t)msgSizeBytes - msgBufNext, (size_t)(end - next)); std::memcpy(msgBuf.get() + msgBufNext, next, bytes2read); @@ -490,7 +477,6 @@ std::tuple readHttpFileAndMergeHttp( offset += bytes2read; msgBufNext += bytes2read; if (msgBufNext == msgSizeBytes) { - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e5"); // Done reading message body. msgBufNext = 0; @@ -503,17 +489,8 @@ std::tuple readHttpFileAndMergeHttp( } // Parse and evaluate the message. - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e6"); mergeSuccess = messageIsReady(msgBuf.get(), msgSizeBytes, last); - LOGS(_log, LOG_LVL_WARN, - context + "&&& readHttpFileAndMergeHttp e7 next=" - << (uint64_t)next << " end=" << (uint64_t)end << " last=" << last - << " success=" << success); totalBytesRead += msgSizeBytes; - LOGS(_log, LOG_LVL_WARN, - context + "&&& readHttpFileAndMergeHttp e7 headerCount=" - << headerCount << " msgSizeBytes=" << msgSizeBytes - << " totalBytesRead=" << totalBytesRead); if (!mergeSuccess) { success = false; throw runtime_error("message processing failed at offset " + @@ -521,49 +498,41 @@ std::tuple readHttpFileAndMergeHttp( } // Reset the variable to prepare for reading the next header & message (if any). msgSizeBytes = 0; - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e8"); } else { LOGS(_log, LOG_LVL_WARN, - "&&&uj headerCount=" << headerCount - << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); + context << " headerCount=" << headerCount + << " incomplete read diff=" << (msgSizeBytes - msgBufNext)); } } } }); - LOGS(_log, LOG_LVL_WARN, - context + "&&& readHttpFileAndMergeHttp e9 headerCount=" - << headerCount << " msgSizeBytes=" << msgSizeBytes + 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); } - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e10"); if (msgBufNext != 0) { throw runtime_error("short read of the message body at offset " + to_string(offset - msgSizeBytes) + ", file: " + httpUrl); } - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp e11"); - } catch (exception const& ex) { // &&&uj anything being caught here besides runtime_error? + } catch (exception const& ex) { LOGS(_log, LOG_LVL_ERROR, context + " " + ex.what()); success = false; } - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp f"); // 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); - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp g"); remover.read([](char const* inBuf, size_t inBufSize) {}); - LOGS(_log, LOG_LVL_WARN, "&&& readHttpFileAndMergeHttp h"); } 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_WARN, - "&&& readHttpFileAndMergeHttp end succes=" << success << " mergeSuccess=" << mergeSuccess); + LOGS(_log, LOG_LVL_DEBUG, context << " end succes=" << success << " mergeSuccess=" << mergeSuccess); return {success, mergeSuccess}; } @@ -726,8 +695,6 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe uint64_t& resultRows) { bool success = false; bool shouldCancel = false; - /// &&&uj NEED CODE - //&&& _wName = responseSummary.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. @@ -745,7 +712,7 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe // 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_WARN, "&&& dataMergerHttp "); + LOGS(_log, LOG_LVL_TRACE, "dataMergerHttp"); last = true; proto::ResponseData responseData; if (responseData.ParseFromArray(buf, bufSize) && responseData.IsInitialized()) { @@ -762,8 +729,9 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe tie(success, shouldCancel) = ::readHttpFileAndMergeHttp(uberJob, fileUrl, dataMergerHttp, MergingHandler::_getHttpConnPool()); - LOGS(_log, LOG_LVL_WARN, - "&&& MergingHandler::flushHttp success=" << success << " shouldCancel=" << shouldCancel); + if (!success || shouldCancel) { + LOGS(_log, LOG_LVL_WARN, __func__ << " success=" << success << " shouldCancel=" << shouldCancel); + } if (success) { _infileMerger->mergeCompleteFor(uberJob->getJobId()); @@ -772,13 +740,6 @@ tuple MergingHandler::flushHttp(string const& fileUrl, uint64_t expe } void MergingHandler::flushHttpError(int errorCode, std::string const& errorMsg, int status) { - /* &&& - _error = util::Error(responseSummary.errorcode(), responseSummary.errormsg(), - util::ErrorCode::MYSQLEXEC); - _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); - _flushError(jq); - */ - if (!_errorSet.exchange(true)) { _error = util::Error(errorCode, errorMsg, util::ErrorCode::MYSQLEXEC); _setError(ccontrol::MSG_RESULT_ERROR, _error.getMsg()); diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index edc5f5a68b..1152dc9324 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -75,15 +75,12 @@ class MergingHandler : public qdisp::ResponseHandler { /// @return true if successful (no error) bool flush(proto::ResponseSummary const& responseSummary, uint32_t& resultRows) override; - /// &&&uj doc see ResponseHandler::flushHttp - /// @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. + /// @see ResponseHandler::flushHttp + /// @see MerginHandler::_mergeHttp std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, uint64_t& resultRows) override; - /// &&&uj doc + /// @see ResponseHandler::flushHttpError void flushHttpError(int errorCode, std::string const& errorMsg, int status) override; /// Signal an unrecoverable error condition. No further calls are expected. @@ -113,7 +110,7 @@ class MergingHandler : public qdisp::ResponseHandler { bool _merge(proto::ResponseSummary const& responseSummary, proto::ResponseData const& responseData, std::shared_ptr const& jobQuery); - /// &&&uj doc + /// 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. @@ -130,7 +127,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}; ///< &&& doc + 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/UserQueryFactory.cc b/src/ccontrol/UserQueryFactory.cc index d3fe5bce5e..85ba8a7dcd 100644 --- a/src/ccontrol/UserQueryFactory.cc +++ b/src/ccontrol/UserQueryFactory.cc @@ -225,9 +225,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 - // &&&uj - std::string stripped; bool async = false; if (UserQueryType::isSubmit(query, stripped)) { diff --git a/src/ccontrol/UserQuerySelect.cc b/src/ccontrol/UserQuerySelect.cc index 2300055e48..f96a293cc0 100644 --- a/src/ccontrol/UserQuerySelect.cc +++ b/src/ccontrol/UserQuerySelect.cc @@ -241,10 +241,8 @@ std::string UserQuerySelect::getResultQuery() const { } /// Begin running on all chunks added so far. -void UserQuerySelect::submit() { //&&&uj - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew start"); +void UserQuerySelect::submit() { _qSession->finalize(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew a"); // Using the QuerySession, generate query specs (text, db, chunkId) and then // create query messages and send them to the async query manager. @@ -253,12 +251,10 @@ void UserQuerySelect::submit() { //&&&uj auto taskMsgFactory = std::make_shared(); _ttn = std::make_shared(_qMetaQueryId, _qSession->getOriginal()); - //&&&_executive->setTmpTableNameGenerator(ttn); std::vector chunks; std::mutex chunksMtx; JobId sequence = 0; - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew b"); auto queryTemplates = _qSession->makeQueryTemplates(); LOGS(_log, LOG_LVL_DEBUG, @@ -266,7 +262,6 @@ void UserQuerySelect::submit() { //&&&uj : "none produced.")); // Writing query for each chunk, stop if query is cancelled. - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew c"); // Add QStatsTmp table entry try { _queryStatsData->queryStatsTmpRegister(_qMetaQueryId, _qSession->getChunksSize()); @@ -279,10 +274,8 @@ void UserQuerySelect::submit() { //&&&uj string dbName(""); bool dbNameSet = false; - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d"); for (auto i = _qSession->cQueryBegin(), e = _qSession->cQueryEnd(); i != e && !_executive->getCancelled(); ++i) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d1"); auto& chunkSpec = *i; // Make the JobQuery now @@ -296,18 +289,16 @@ void UserQuerySelect::submit() { //&&&uj } std::string chunkResultName = _ttn->make(cs->chunkId); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d2"); // This should only need to be set once as all jobs should have the same database name. if (cs->db != dbName) { - LOGS(_log, LOG_LVL_WARN, "&&& dbName change from " << dbName << " to " << cs->db); if (dbNameSet) { - throw util::Bug(ERR_LOC, "Multiple database names in UberJob"); + LOGS(_log, LOG_LVL_ERROR, "dbName change from " << dbName << " to " << cs->db); + return; } dbName = cs->db; dbNameSet = true; } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d3"); ResourceUnit ru; ru.setAsDbChunk(cs->db, cs->chunkId); qdisp::JobDescription::Ptr jobDesc = qdisp::JobDescription::create( @@ -316,21 +307,14 @@ void UserQuerySelect::submit() { //&&&uj chunkResultName); auto job = _executive->add(jobDesc); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4"); if (!uberJobsEnabled) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4a"); // references in captures cause races auto funcBuildJob = [this, job{move(job)}](util::CmdData*) { QSERV_LOGCONTEXT_QUERY(_qMetaQueryId); - //&&& job->runJob(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew lambda a"); _executive->runJobQuery(job); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew lambda b"); }; auto cmd = std::make_shared(funcBuildJob); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4b"); _executive->queueJobStart(cmd); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew d4c"); } ++sequence; } @@ -339,46 +323,36 @@ void UserQuerySelect::submit() { //&&&uj _queryDbName = dbName; } - /// &&& ******************************************************** - /// &&&uj at this point the executive has a map of all jobs with the chunkIds as the key. - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e"); - //&&&if (uberJobsEnabled || true) { + /// At this point the executive has a map of all jobs with the chunkIds as the key. if (uberJobsEnabled) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1"); - _maxChunksPerUberJob = 2; // &&&uj maybe put in config??? or set on command line?? - // &&&uj Different queries may benefit from different values - // &&&uj Such as LIMIT=1 may work best with this at 1, where - // &&&uj 100 would be better for others. - //&&&_executive->buildAndSendUberJobs(maxChunksPerUber); + // 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_WARN, "&&& UserQuerySelect::submitNew e2"); } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); - _executive->waitForAllJobsToStart(); // &&& this may not be needed anymore? + // TODO:UJ Waiting for all jobs to start may not be needed anymore? + _executive->waitForAllJobsToStart(); // we only care about per-chunk info for ASYNC queries if (_async) { std::lock_guard lock(chunksMtx); _qMetaAddChunks(chunks); } - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew end"); } void UserQuerySelect::buildAndSendUberJobs() { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs a"); string const funcN("UserQuerySelect::" + string(__func__) + " QID=" + to_string(_qMetaQueryId)); - LOGS(_log, LOG_LVL_INFO, funcN << " start"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start"); // Ensure `_monitor()` doesn't do anything until everything is ready. if (!_executive->isReadyToExecute()) { - LOGS(_log, LOG_LVL_DEBUG, - "UserQuerySelect::" << __func__ << " executive isn't ready to generate UberJobs."); + LOGS(_log, LOG_LVL_INFO, funcN << " executive isn't ready to generate UberJobs."); return; } @@ -386,13 +360,11 @@ void UserQuerySelect::buildAndSendUberJobs() { lock_guard fcLock(_buildUberJobMtx); bool const clearFlag = false; _executive->setFlagFailedUberJob(clearFlag); - LOGS(_log, LOG_LVL_WARN, - "&&& UserQuerySelect::buildAndSendUberJobs totalJobs=" << _executive->getTotalJobs()); + LOGS(_log, LOG_LVL_DEBUG, "UserQuerySelect::" << __func__ << " totalJobs=" << _executive->getTotalJobs()); vector uberJobs; auto czarPtr = czar::Czar::getCzar(); - //&&&auto czChunkMap = czarPtr->getCzarChunkMap(); auto czFamilyMap = czarPtr->getCzarFamilyMap(); auto czChunkMap = czFamilyMap->getChunkMap(_queryDbName); auto czRegistry = czarPtr->getCzarRegistry(); @@ -402,39 +374,15 @@ void UserQuerySelect::buildAndSendUberJobs() { // 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(); - // TODO:UJ It may be better to just fail the query now, but with a working - // system, this should be very rare. } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs b"); - auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs b1"); - + auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); // Make a map of all jobs in the executive. - // &&& TODO:UJ At some point, need to check that ResourceUnit databases can - // &&& be found for all databases in the query. - // &&& NEED CODE to use database family instead of making this check. + // TODO:UJ Maybe a check should be made that all datbases are in the same family? qdisp::Executive::ChunkIdJobMapType unassignedChunksInQuery = _executive->unassignedChunksInQuery(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs c"); - - // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use - // the same databases) Use this to check for conflicts - - /* &&& - // assign jobs to uberJobs - int maxChunksPerUber = 3; // &&&uj maybe put in config??? or set on command line?? - // &&&uj Different queries may benefit from different values - // &&&uj Such as LIMIT=1 may work best with this at 1, where - // &&&uj 100 would be better for others. - */ // keep cycling through workers until no more chunks to place. - - // TODO:UJ &&&uj Once everything is an UberJob, it can start with 1 or 0. - // int _uberJobId = qdisp::UberJob::getFirstIdNumber(); - - // &&&uj // - create a map of UberJobs key=, val=> // - for chunkId in `unassignedChunksInQuery` // - use `chunkMapPtr` to find the shared scan workerId for chunkId @@ -457,24 +405,19 @@ void UserQuerySelect::buildAndSendUberJobs() { // 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. - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d"); for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d1"); auto iter = chunkMapPtr->find(chunkId); if (iter == chunkMapPtr->end()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d1a"); 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; } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d2"); czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d3"); - if (targetWorker == - nullptr) { //&&&uj if (targetWorker == nullptr || this worker already tried for this chunk) { + // 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(); @@ -490,24 +433,20 @@ void UserQuerySelect::buildAndSendUberJobs() { } } if (!found) { - // &&&uj If too many workers are down, there will be a chunk that cannot be found. - // the correct course of action is probably to check the Registry, and - // after so many attempts, cancel the user query with a - // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount - // in the Job or JobDescription could be used for this. + // 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()); - throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + - " Crashing the program here for this reason is not " - "appropriate. &&& NEEDS CODE"); + continue; } } // Add this job to the appropriate UberJob, making the UberJob if needed. string workerId = targetWorker->getWorkerId(); auto& ujVect = workerJobMap[workerId]; - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d4 ujVect.sz=" << ujVect.size()); if (ujVect.empty() || ujVect.back()->getJobCount() >= _maxChunksPerUberJob) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d4a"); auto ujId = _uberJobIdSeq++; // keep ujId consistent string uberResultName = _ttn->make(ujId); auto respHandler = make_shared(_infileMerger, uberResultName); @@ -516,19 +455,13 @@ void UserQuerySelect::buildAndSendUberJobs() { ujVect.push_back(uJob); } auto& ujVectBack = ujVect.back(); - LOGS(_log, LOG_LVL_WARN, - "&&& UserQuerySelect::buildAndSendUberJobs d4b ujVectBack{" - << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() << "}"); ujVectBack->addJob(jqPtr); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d5 ujVect.sz=" << ujVect.size()); - LOGS(_log, LOG_LVL_WARN, - "&&& UserQuerySelect::buildAndSendUberJobs d5a ujVectBack{" - << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() << "}"); + LOGS(_log, LOG_LVL_DEBUG, + funcN << " ujVectBack{" << ujVectBack->getIdStr() << " jobCnt=" << ujVectBack->getJobCount() + << "}"); } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d6"); if (!missingChunks.empty()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs d6a"); string errStr = funcN + " a worker could not be found for these chunks "; for (auto const& chk : missingChunks) { errStr += to_string(chk) + ","; @@ -539,39 +472,27 @@ void UserQuerySelect::buildAndSendUberJobs() { // new uber jobs for these jobs. _executive->setFlagFailedUberJob(true); } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs e"); - //&&&uj // Add worker contact info to UberJobs. auto const wContactMap = czRegistry->getWorkerContactMap(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f"); - LOGS(_log, LOG_LVL_WARN, - "&&& UserQuerySelect::buildAndSendUberJobs f" << _executive->dumpUberJobCounts()); + LOGS(_log, LOG_LVL_DEBUG, funcN << " " << _executive->dumpUberJobCounts()); for (auto const& [wIdKey, ujVect] : workerJobMap) { - LOGS(_log, LOG_LVL_WARN, - "&&& UserQuerySelect::buildAndSendUberJobs f1 wId=" << wIdKey << " ujVect.sz=" << ujVect.size()); auto iter = wContactMap->find(wIdKey); if (iter == wContactMap->end()) { - // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. - throw util::Bug(ERR_LOC, funcN + " &&&uj NEED CODE, no contact information for " + wIdKey); + // 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) { - LOGS(_log, LOG_LVL_WARN, - ujPtr->getIdStr() << " " << wContactInfo->dump() - << " &&& UserQuerySelect::buildAndSendUberJobs f2"); ujPtr->setWorkerContactInfo(wContactInfo); } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f3"); _executive->addUberJobs(ujVect); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f4"); for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::buildAndSendUberJobs f5"); _executive->runUberJob(ujPtr); } } - LOGS(_log, LOG_LVL_WARN, - "&&& UserQuerySelect::buildAndSendUberJobs g" << _executive->dumpUberJobCounts()); } /// Block until a submit()'ed query completes. diff --git a/src/ccontrol/UserQuerySelect.h b/src/ccontrol/UserQuerySelect.h index b346d92030..a01b973cd8 100644 --- a/src/ccontrol/UserQuerySelect.h +++ b/src/ccontrol/UserQuerySelect.h @@ -154,7 +154,9 @@ class UserQuerySelect : public UserQuery { /// save the result query in the query metadata void saveResultQuery(); - /// &&&uj doc + /// 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: @@ -191,19 +193,23 @@ class UserQuerySelect : public UserQuery { std::string _queryIdStr{QueryIdHelper::makeIdStr(0, true)}; bool _killed{false}; std::mutex _killMutex; - 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??) - bool _async; ///< true for async query - int _maxChunksPerUberJob = 1; ///< &&&uj - std::atomic _uberJobIdSeq{900'000}; ///< &&&uj can probably start at 1 - std::shared_ptr _ttn; ///< Temporary table name generator. + mutable std::string _errorExtra; ///< Additional error information + std::string _resultTable; ///< Result table name + std::string _resultLoc; ///< Result location + 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; - /// &&&uj Only one thread should run buildAndSendUberJobs() at a time. + /// Only one thread should run buildAndSendUberJobs() for this query at a time. std::mutex _buildUberJobMtx; }; diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 73a686a099..f8434281c0 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -169,14 +169,12 @@ Czar::Ptr Czar::createCzar(string const& configFilePath, string const& czarName) } void Czar::_monitor() { - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor a"); + string const funcN("Czar::_monitor"); while (_monitorLoop) { - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor b"); this_thread::sleep_for(_monitorSleepTime); - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor c"); + LOGS(_log, LOG_LVL_DEBUG, funcN << " start0"); /// Check database for changes in worker chunk assignments and aliveness - //&&&_czarChunkMap->read(); _czarFamilyMap->read(); // TODO:UJ If there were changes in `_czarFamilyMap`, see if any @@ -184,8 +182,8 @@ void Czar::_monitor() { // for the downed workers. The `_unassigned` Jobs should get // reassigned in the next section `assignJobsToUberJobs`. - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor d"); - /// Create new UberJobs for all jobs that are unassigned for any reason. + /// Create new UberJobs (if possible) for all jobs that are + /// unassigned for any reason. map> execMap; { // Make a copy of all valid Executives @@ -202,12 +200,10 @@ void Czar::_monitor() { } } } - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor e"); // Use the copy to create new UberJobs as needed for (auto&& [qIdKey, execVal] : execMap) { execVal->assignJobsToUberJobs(); } - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor f"); // TODO:UJ Maybe get missing results from workers. // This would be files that workers sent messages to the czar to @@ -220,7 +216,6 @@ void Czar::_monitor() { // TODO:UJ Maybe send a list of cancelled and completed queries to the workers? // How long should queryId's remain on this list? } - LOGS(_log, LOG_LVL_WARN, "&&& Czar::_monitor end"); } // Constructors @@ -247,7 +242,6 @@ Czar::Czar(string const& configFilePath, string const& czarName) _czarConfig->setId(_uqFactory->userQuerySharedResources()->qMetaCzarId); try { - //&&& _czarChunkMap = CzarChunkMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); _czarFamilyMap = CzarFamilyMap::create(_uqFactory->userQuerySharedResources()->queryMetadata); } catch (ChunkMapException const& exc) { LOGS(_log, LOG_LVL_WARN, string(__func__) + " failed to create CzarChunkMap " + exc.what()); @@ -404,28 +398,20 @@ SubmitResult Czar::submitQuery(string const& query, map const& h // note that lambda stores copies of uq and msgTable. auto finalizer = [uq, msgTable]() mutable { string qidstr = to_string(uq->getQueryId()); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " a"); // Add logging context with query ID QSERV_LOGCONTEXT_QUERY(uq->getQueryId()); LOGS(_log, LOG_LVL_DEBUG, "submitting new query"); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " b"); uq->submit(); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " c"); uq->join(); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " d"); try { msgTable.unlock(uq); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " e"); if (uq) uq->discard(); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " f"); } catch (std::exception const& exc) { // TODO? if this fails there is no way to notify client, and client // will likely hang because table may still be locked. LOGS(_log, LOG_LVL_ERROR, "Query finalization failed (client likely hangs): " << exc.what()); } - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " g"); uq.reset(); - LOGS(_log, LOG_LVL_WARN, "&&& finalizer QID=" << qidstr << " end"); }; LOGS(_log, LOG_LVL_DEBUG, "starting finalizer thread for query"); thread finalThread(finalizer); diff --git a/src/czar/Czar.h b/src/czar/Czar.h index 74eacdb5f4..9a39eaccee 100644 --- a/src/czar/Czar.h +++ b/src/czar/Czar.h @@ -137,10 +137,10 @@ class Czar { std::shared_ptr getCzarRegistry() const { return _czarRegistry; } - /// &&& doc + /// Add an Executive to the map of executives. void insertExecutive(QueryId qId, std::shared_ptr const& execPtr); - /// &&& doc + /// Get the executive associated with `qId`, this may be nullptr. std::shared_ptr getExecutiveFromMap(QueryId qId); private: @@ -162,7 +162,7 @@ class Czar { /// @return An identifier of the last query that was recorded in the query metadata table QueryId _lastQueryIdBeforeRestart() const; - /// &&& doc + /// 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. @@ -202,7 +202,6 @@ class Czar { std::shared_ptr _controlHttpSvc; /// Map of which chunks on which workers and shared scan order. - // &&& std::shared_ptr _czarChunkMap; std::shared_ptr _czarFamilyMap; /// Connection to the registry to register the czar and get worker contact information. @@ -212,9 +211,12 @@ class Czar { std::map> _executiveMap; ///< Map of executives for queries in progress. - std::thread _monitorThrd; ///< &&& doc - std::atomic _monitorLoop{true}; ///< &&& doc - std::chrono::milliseconds _monitorSleepTime{15000}; ///< Wait time between checks. &&& set from config + 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 index 59c0e28713..2570b3dc9a 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -45,195 +45,10 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.CzarChunkMap"); namespace lsst::qserv::czar { -/* &&& -CzarChunkMap::CzarChunkMap(std::shared_ptr const& qmeta) : _qmeta(qmeta) { - try { - auto mapsSet = _readOld(); - if (!mapsSet) { - throw ChunkMapException(ERR_LOC, "CzarChunkMap maps were not set in contructor"); - } - } catch (qmeta::QMetaError const& qExc) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " CzarChunkMap could not read DB " << qExc.what()); - throw ChunkMapException(ERR_LOC, string(" CzarChunkMap constructor failed read ") + qExc.what()); - } -} -*/ - CzarChunkMap::CzarChunkMap() {} CzarChunkMap::~CzarChunkMap() { LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap::~CzarChunkMap()"); } -/* &&& -bool CzarChunkMap::readOld() { - bool mapsSet = false; - try { - mapsSet = _readOld(); - } catch (qmeta::QMetaError const& qExc) { - LOGS(_log, LOG_LVL_ERROR, __func__ << " CzarChunkMap could not read DB " << qExc.what()); - } - return mapsSet; -} - -bool CzarChunkMap::_readOld() { - LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_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(_mapMtx); - qmeta::QMetaChunkMap qChunkMap = _qmeta->getChunkMap(); - if (_lastUpdateTime >= qChunkMap.updateTime) { - LOGS(_log, LOG_LVL_DEBUG, - __func__ << " CzarChunkMap no need to read " - << util::TimeUtils::timePointToDateTimeString(_lastUpdateTime) - << " db=" << util::TimeUtils::timePointToDateTimeString(qChunkMap.updateTime)); - return false; - } - - // Make the new maps. - auto [chunkMapPtr, wcMapPtr] = makeNewMapsOld(qChunkMap); - - verifyOld(*chunkMapPtr, *wcMapPtr); - - LOGS(_log, LOG_LVL_DEBUG, " chunkMap=" << dumpChunkMap(*chunkMapPtr)); - LOGS(_log, LOG_LVL_DEBUG, " workerChunkMap=" << dumpWorkerChunkMap(*wcMapPtr)); - - _workerChunkMap = wcMapPtr; - _chunkMap = chunkMapPtr; - //&&&_lastUpdateTime = qChunkMap.updateTime; - - LOGS(_log, LOG_LVL_TRACE, "CzarChunkMap::_read() end"); - return true; -} -*/ - -pair, shared_ptr> -CzarChunkMap::makeNewMapsOld(qmeta::QMetaChunkMap const& qChunkMap) { - // Create new maps. - auto wcMapPtr = make_shared(); - auto chunkMapPtr = make_shared(); - - // Workers -> Databases map - 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; - SizeT sz = chunkInfo.size; - LOGS(_log, LOG_LVL_DEBUG, - "workerdId=" << workerId << " db=" << dbName << " table=" << tableName - << " chunk=" << chunkNum << " sz=" << sz); - insertIntoChunkMap(*wcMapPtr, *chunkMapPtr, workerId, dbName, tableName, chunkNum, - sz); - } catch (invalid_argument const& exc) { - throw ChunkMapException( - ERR_LOC, string(__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, string(__func__) + " out_of_range workerdId=" + workerId + - " db=" + dbName + " table=" + tableName + - " chunk=" + to_string(chunkInfo.chunk) + " " + exc.what()); - } - } - } - } - } - - auto chunksSortedBySize = make_shared(); - calcChunkMap(*chunkMapPtr, *chunksSortedBySize); - - // At this point we have - // - wcMapPtr has a map of workerData by worker id with each worker having a map of ChunkData - // - chunkMapPtr 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 ok. - } - 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); - } - - LOGS(_log, LOG_LVL_TRACE, " chunkMap=" << dumpChunkMap(*chunkMapPtr)); - LOGS(_log, LOG_LVL_TRACE, " workerChunkMap=" << dumpWorkerChunkMap(*wcMapPtr)); - return {chunkMapPtr, wcMapPtr}; -} - -void CzarChunkMap::insertIntoChunkMap(WorkerChunkMap& wcMap, ChunkMap& chunkMap, string const& workerId, - string const& dbName, string const& tableName, int64_t chunkIdNum, - SizeT sz) { - // Get or make the worker entry - WorkerChunksData::Ptr workerChunksData; - auto iterWC = wcMap.find(workerId); - if (iterWC == wcMap.end()) { - workerChunksData = WorkerChunksData::Ptr(new WorkerChunksData(workerId)); - wcMap[workerId] = workerChunksData; - } else { - workerChunksData = iterWC->second; - } - - // Get or make the ChunkData entry in chunkMap - ChunkData::Ptr chunkData; - auto iterChunkData = chunkMap.find(chunkIdNum); - if (iterChunkData == chunkMap.end()) { - chunkData = ChunkData::Ptr(new 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, - __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 CzarChunkMap::calcChunkMap(ChunkMap const& chunkMap, ChunkVector& chunksSortedBySize) { // Calculate total bytes for all chunks. for (auto&& [chunkIdNum, chunkData] : chunkMap) { @@ -361,7 +176,6 @@ CzarChunkMap::ChunkData::getWorkerHasThisMapCopy() const { void CzarChunkMap::organize() { auto chunksSortedBySize = make_shared(); - //&&&calcChunkMap(*chunkMapPtr, *chunksSortedBySize); calcChunkMap(*_chunkMap, *chunksSortedBySize); // At this point we have @@ -483,174 +297,116 @@ bool CzarFamilyMap::_read() { std::shared_ptr CzarFamilyMap::makeNewMaps( qmeta::QMetaChunkMap const& qChunkMap) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& a"); // Create new maps. std::shared_ptr newFamilyMap = make_shared(); - //&&&auto wcMapPtr = make_shared(); - //&&&auto chunkMapPtr = make_shared(); // Workers -> Databases map - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& b workers.sz=" << qChunkMap.workers.size()); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " workers.sz=" << qChunkMap.workers.size()); for (auto const& [workerId, dbs] : qChunkMap.workers) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& c " << workerId << " dbs.sz=" << dbs.size()); // Databases -> Tables map for (auto const& [dbName, tables] : dbs) { - LOGS(_log, LOG_LVL_WARN, - "CzarFamilyMap::makeNewMaps &&& d " << dbName << " tbls.sz=" << tables.size()); // Tables -> Chunks map for (auto const& [tableName, chunks] : tables) { - LOGS(_log, LOG_LVL_WARN, - "CzarFamilyMap::makeNewMaps &&& e " << tableName << " chunks.sz=" << chunks.size()); // vector of ChunkInfo for (qmeta::QMetaChunkMap::ChunkInfo const& chunkInfo : chunks) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& f"); try { int64_t chunkNum = chunkInfo.chunk; CzarChunkMap::SizeT sz = chunkInfo.size; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& g"); LOGS(_log, LOG_LVL_DEBUG, - "workerdId=" << workerId << " db=" << dbName << " table=" << tableName - << " chunk=" << chunkNum << " sz=" << sz); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& h"); + cName(__func__) << "workerdId=" << workerId << " db=" << dbName << " table=" + << tableName << " chunk=" << chunkNum << " sz=" << sz); insertIntoMaps(newFamilyMap, workerId, dbName, tableName, chunkNum, sz); } catch (invalid_argument const& exc) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& h1"); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& EXCEPTION a"); throw ChunkMapException( - ERR_LOC, string(__func__) + " invalid_argument workerdId=" + workerId + + 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) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& h2"); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& EXCEPTION b"); throw ChunkMapException( - ERR_LOC, string(__func__) + " out_of_range workerdId=" + workerId + + ERR_LOC, cName(__func__) + " out_of_range workerdId=" + workerId + " db=" + dbName + " table=" + tableName + " chunk=" + to_string(chunkInfo.chunk) + " " + exc.what()); } } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& e " << tableName << " end"); } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& d " << dbName << " end"); } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& c " << workerId << " end"); } // this needs to be done for each CzarChunkMap in the family map. - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& i"); for (auto&& [familyName, chunkMapPtr] : *newFamilyMap) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& j"); - LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap::makeNewMaps working on " << familyName); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& k"); - + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " working on " << familyName); chunkMapPtr->organize(); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& l"); } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::makeNewMaps &&& end"); 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) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& a"); // Get the CzarChunkMap for this family auto familyName = getFamilyNameFromDbName(dbName); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& b"); 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; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c"); auto familyIter = nfMap.find(familyName); if (familyIter == nfMap.end()) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c1"); czarChunkMap = CzarChunkMap::Ptr(new CzarChunkMap()); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c1a"); nfMap[familyName] = czarChunkMap; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c1b"); } else { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c2"); czarChunkMap = familyIter->second; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& c2a"); } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& d"); auto [chunkMapPtr, wcMapPtr] = czarChunkMap->_getMaps(); - // &&& This bit of indirection is both no longer needed and confusing. - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& e"); CzarChunkMap::WorkerChunkMap& wcMap = *wcMapPtr; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& f"); CzarChunkMap::ChunkMap& chunkMap = *chunkMapPtr; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& g"); // Get or make the worker entry CzarChunkMap::WorkerChunksData::Ptr workerChunksData; auto iterWC = wcMap.find(workerId); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h"); if (iterWC == wcMap.end()) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h1"); workerChunksData = CzarChunkMap::WorkerChunksData::Ptr(new CzarChunkMap::WorkerChunksData(workerId)); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h1a"); wcMap[workerId] = workerChunksData; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h1b"); } else { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& h2"); workerChunksData = iterWC->second; } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& i"); // Get or make the ChunkData entry in chunkMap CzarChunkMap::ChunkData::Ptr chunkData; auto iterChunkData = chunkMap.find(chunkIdNum); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j"); if (iterChunkData == chunkMap.end()) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j1"); chunkData = CzarChunkMap::ChunkData::Ptr(new CzarChunkMap::ChunkData(chunkIdNum)); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j1a"); chunkMap[chunkIdNum] = chunkData; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j1b"); } else { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& j2"); chunkData = iterChunkData->second; } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& k"); // Set or verify the table information auto iterDT = chunkData->_dbTableMap.find({dbName, tableName}); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l"); if (iterDT == chunkData->_dbTableMap.end()) { // doesn't exist so set it up - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l1"); chunkData->_dbTableMap[{dbName, tableName}] = sz; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l1a"); } else { // Verify that it matches other data - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l2"); auto const& dbTbl = iterDT->first; auto tblSz = iterDT->second; auto const& dbN = dbTbl.first; auto const& tblN = dbTbl.second; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l2a"); if (dbName != dbN || tblN != tableName || tblSz != sz) { - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& l2a1"); LOGS(_log, LOG_LVL_ERROR, - __func__ << " data mismatch for " << dbName << "." << tableName << "=" << sz << " vs " << dbN - << "." << tblN << "=" << tblSz); + cName(__func__) << " data mismatch for " << dbName << "." << tableName << "=" << sz << " vs " + << dbN << "." << tblN << "=" << tblSz); } } - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& m"); // Link WorkerData the single chunkData instance for the chunkId workerChunksData->_chunkDataMap[chunkIdNum] = chunkData; - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& n"); // Add worker to the list of workers containing the chunk. chunkData->addToWorkerHasThis(workerChunksData); - LOGS(_log, LOG_LVL_WARN, "CzarFamilyMap::insertIntoMaps &&& end"); } void CzarFamilyMap::verify(std::shared_ptr const& familyMap) { diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index de9f8a6095..b6f69d38b2 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -50,22 +50,6 @@ class ChunkMapException : public util::Issue { ChunkMapException(Context const& ctx, std::string const& msg) : util::Issue(ctx, msg) {} }; -// &&&&&&&&&& Provide a map based on family name (which will be based on database name for now) -// to determine which maps to use. 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. -/// 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 @@ -87,7 +71,6 @@ class CzarChunkMap { using Ptr = std::shared_ptr; using SizeT = uint64_t; - //&&&CzarChunkMap() = delete; CzarChunkMap(CzarChunkMap const&) = delete; CzarChunkMap& operator=(CzarChunkMap const&) = delete; @@ -185,36 +168,15 @@ class CzarChunkMap { /// Sort the chunks in `chunksSortedBySize` in descending order by total size in bytes. static void sortChunks(ChunkVector& chunksSortedBySize); - /// Insert the chunk table described into the correct locations in - /// `wcMap` and `chunkMap`. - /// @param `wcMap` - WorkerChunkMap being constructed. - /// @param `chunkMap` - ChunkMap being constructed. - /// @param `workerId` - worker id where this table was found. - /// @param `dbName` - database name for the table being inserted. - /// @param `tableName` - table name for the table being inserted. - /// @param `chunkIdNum` - chunk id number for the table being inserted. - /// @param `sz` - size in bytes of the table being inserted. - static void insertIntoChunkMap(WorkerChunkMap& wcMap, ChunkMap& chunkMap, std::string const& workerId, - std::string const& dbName, std::string const& tableName, - int64_t chunkIdNum, SizeT sz); /// &&& delete - /// 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); - /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. - static std::pair, std::shared_ptr> - makeNewMapsOld(qmeta::QMetaChunkMap const& qChunkMap); //&&& delete - /// Verify that all chunks belong to at least one worker and that all chunks are represented in shared /// scans. /// @throws ChunkMapException - static void verifyOld(ChunkMap const& chunkMap, WorkerChunkMap const& wcMap); // &&& delete void verify(); - // void setMaps(std::shared_ptr const& chunkMapPtr, std::shared_ptr const& - // wcMapPtr); // &&& delete - static std::string dumpChunkMap(ChunkMap const& chunkMap); static std::string dumpWorkerChunkMap(WorkerChunkMap const& wcMap); @@ -228,17 +190,11 @@ class CzarChunkMap { return {_chunkMap, _workerChunkMap}; } - /// &&& doc + /// Use the information from the registry to `organize` `_chunkMap` and `_workerChunkMap` + /// into their expected formats. void organize(); - /// &&& doc - //&&&bool readOld(); - private: - /* &&& - /// Try to `_read` values for maps from `qmeta`. - CzarChunkMap(std::shared_ptr const& qmeta); - */ CzarChunkMap(); /// Return shared pointers to `_chunkMap` and `_workerChunkMap`, which should be held until @@ -249,13 +205,6 @@ class CzarChunkMap { return {_chunkMap, _workerChunkMap}; } - /// Read the json worker list from the database and update the maps if there's a new - /// version since the `_lastUpdateTime`. - /// @throws `qmeta::QMetaError` - //&&& bool _readOld(); - - //&&&std::shared_ptr _qmeta; ///< Database connection to collect json worker list. - /// Map of all workers and which chunks they contain. std::shared_ptr _workerChunkMap{new WorkerChunkMap()}; @@ -264,16 +213,31 @@ class CzarChunkMap { /// handling the chunk in a shared scan. std::shared_ptr _chunkMap{new ChunkMap()}; - /* &&& - /// The last time the maps were updated with information from the replicator. - TIMEPOINT _lastUpdateTime; // initialized to 0; - */ - mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap (&&& still needed???) + mutable std::mutex _mapMtx; ///< protects _workerChunkMap, _chunkMap (TODO:UJ may not be needed anymore) friend CzarFamilyMap; }; -/// &&& doc +/// 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; @@ -311,28 +275,29 @@ class CzarFamilyMap { return _getChunkMap(familyName); } - /// &&& doc + /// Read the registry information from the database, if not already set. bool read(); - /// &&& doc - /// Make new ChunkMap and WorkerChunkMap from the data in `qChunkMap`. - //&&&static std::pair, - //std::shared_ptr> + /// 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); - // &&& doc + /// 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); - /// &&& doc + /// 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` diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index a76add0ba1..3d179d577f 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -61,10 +61,7 @@ HttpCzarWorkerModule::HttpCzarWorkerModule(string const& context, shared_ptrreplicationInstanceId()); enforceCzarName(func); if (subModuleName == "QUERYJOB-ERROR") return _queryJobError(); @@ -76,8 +73,7 @@ json HttpCzarWorkerModule::executeImpl(string const& subModuleName) { json HttpCzarWorkerModule::_queryJobError() { debug(__func__); checkApiVersion(__func__, 34); - LOGS(_log, LOG_LVL_INFO, __func__ << "&&&uj queryJobError json=" << body().objJson); //&&& - //&&&uj NEED CODE for this + LOGS(_log, LOG_LVL_DEBUG, __func__ << " queryJobError json=" << body().objJson); auto ret = _handleJobError(__func__); return json::object(); } @@ -85,12 +81,9 @@ json HttpCzarWorkerModule::_queryJobError() { json HttpCzarWorkerModule::_queryJobReady() { debug(__func__); checkApiVersion(__func__, 34); - LOGS(_log, LOG_LVL_INFO, __func__ << "&&&uj queryJobReady json=" << body().objJson); //&&& + LOGS(_log, LOG_LVL_DEBUG, __func__ << " queryJobReady json=" << body().objJson); auto ret = _handleJobReady(__func__); return ret; - - //&&& json ret = {{"success", 1}}; - //&&&return json::object(); } json HttpCzarWorkerModule::_handleJobError(string const& func) { @@ -100,35 +93,26 @@ json HttpCzarWorkerModule::_handleJobError(string const& func) { json jsRet = {{"success", 0}, {"errortype", "unknown"}, {"note", "initialized"}}; try { // See qdisp::UberJob::runUberJob() for json message construction. - auto const& js = body().objJson; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR js=" << js); string const targetWorkerId = body().required("workerid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR targetWorkerId=" << targetWorkerId); string const czarName = body().required("czar"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarName=" << czarName); qmeta::CzarId const czarId = body().required("czarid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarId=" << czarId); QueryId const queryId = body().required("queryid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR queryId=" << queryId); UberJobId const uberJobId = body().required("uberjobid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR uberJobId=" << uberJobId); int const errorCode = body().required("errorCode"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR errorCode=" << errorCode); string const errorMsg = body().required("errorMsg"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR errorMsg=" << 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)); + 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)); + to_string(queryId) + " ujId=" + to_string(uberJobId) + + " czar=" + to_string(czarId)); } - // &&&uj NEED CODE to verify incoming values to those in the UberJob auto importRes = uj->workerError(errorCode, errorMsg); jsRet = importRes; @@ -148,37 +132,27 @@ json HttpCzarWorkerModule::_handleJobReady(string const& func) { json jsRet = {{"success", 1}, {"errortype", "unknown"}, {"note", "initialized"}}; try { // See qdisp::UberJob::runUberJob() for json message construction. - auto const& js = body().objJson; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR js=" << js); string const targetWorkerId = body().required("workerid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR targetWorkerId=" << targetWorkerId); string const czarName = body().required("czar"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarName=" << czarName); qmeta::CzarId const czarId = body().required("czarid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR czarId=" << czarId); QueryId const queryId = body().required("queryid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR queryId=" << queryId); UberJobId const uberJobId = body().required("uberjobid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR uberJobId=" << uberJobId); string const fileUrl = body().required("fileUrl"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR fileUrl=" << fileUrl); uint64_t const rowCount = body().required("rowCount"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR rowCount=" << rowCount); uint64_t const fileSize = body().required("fileSize"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&UJR fileSize=" << 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)); + 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)); + to_string(queryId) + " ujId=" + to_string(uberJobId) + + " czar=" + to_string(czarId)); } - // &&&uj NEED CODE to verify incoming values to those in the UberJob auto importRes = uj->importResultFile(fileUrl, rowCount, fileSize); jsRet = importRes; diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h index 9e6fc831a0..ef0a745093 100644 --- a/src/czar/HttpCzarWorkerModule.h +++ b/src/czar/HttpCzarWorkerModule.h @@ -40,7 +40,7 @@ class Response; // This header declarations namespace lsst::qserv::czar { -/// &&& doc This class is used to handle messages to this czar from workers. +/// 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: @@ -64,15 +64,16 @@ class HttpCzarWorkerModule : public czar::HttpModule { HttpCzarWorkerModule(std::string const& context, std::shared_ptr const& req, std::shared_ptr const& resp); - /// &&& doc + /// Called to handle message indicating this czar needs to handle an error on a worker. nlohmann::json _queryJobError(); - /// &&& doc + /// Called to indicate an UberJob is ready with data that needs to be collected. nlohmann::json _queryJobReady(); - /// &&& doc + /// 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); }; diff --git a/src/czar/testCzar.cc b/src/czar/testCzar.cc index 96ee576d4d..aad9fdfd31 100644 --- a/src/czar/testCzar.cc +++ b/src/czar/testCzar.cc @@ -186,26 +186,11 @@ BOOST_AUTO_TEST_CASE(CzarChunkMap) { } )"; - /* &&& - auto jsTest1 = nlohmann::json::parse(test1); - qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); - auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); - czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. - LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 1 passed"); - - auto jsTest2 = nlohmann::json::parse(test2); - qmeta::QMetaChunkMap qChunkMap2 = convertJsonToChunkMap(jsTest2); - tie(chunkMapPtr, wcMapPtr) = czar::CzarChunkMap::makeNewMaps(qChunkMap2); - czar::CzarChunkMap::verify(*chunkMapPtr, *wcMapPtr); // Throws on failure. - LOGS(_log, LOG_LVL_DEBUG, "CzarChunkMap test 2 passed"); - */ - auto dbToFamily = make_shared(); czar::CzarFamilyMap czFamMap(dbToFamily); auto jsTest1 = nlohmann::json::parse(test1); qmeta::QMetaChunkMap qChunkMap1 = convertJsonToChunkMap(jsTest1); - //&&&auto [chunkMapPtr, wcMapPtr] = czar::CzarChunkMap::makeNewMaps(qChunkMap1); auto familyMap = czFamMap.makeNewMaps(qChunkMap1); czar::CzarFamilyMap::verify(familyMap); // Throws on failure. LOGS(_log, LOG_LVL_DEBUG, "CzarFamilyMap test 1 passed"); diff --git a/src/http/RequestBodyJSON.h b/src/http/RequestBodyJSON.h index f82c78a288..fa738e8556 100644 --- a/src/http/RequestBodyJSON.h +++ b/src/http/RequestBodyJSON.h @@ -48,9 +48,9 @@ class RequestBodyJSON { ~RequestBody() = default; - /// &&& doc - /// &&&uj This would be much more efficient if this class had objJson defined as - /// &&&uj a const reference or pointer to const, but implementation likely ugly. + /// 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) {} /** diff --git a/src/proto/worker.proto b/src/proto/worker.proto index d82af83658..76d607997f 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 { @@ -75,18 +76,6 @@ message TaskMsg { optional int32 maxtablesize_mb = 15 [default = 0]; } -//&&&uj To be deleted. Protobuffs will no longer be needed once xrootd is removed. -// TODO:UJ For now, just a pile of task messages. Eventually, there should be a template -// with chunkId numbers which the worker would then expand into full queries. Much -// like how subchunks are now handled. -message UberJobMsg { - required uint64 queryid = 1; - required uint32 czarid = 2; - required uint32 uberjobid = 3; - repeated TaskMsg taskmsgs = 4; - required uint32 magicnumber = 5; -} - // The file-based result delivery protocol has two kinds of messages. // // 1. The summary message sent back to Czar over the XROOTD/SSI protocol: diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index 9d005ac30c..97ebf2d6a0 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -100,12 +100,6 @@ string getErrorText(XrdSsiErrInfo& e) { namespace lsst::qserv::qdisp { -/* &&& -mutex Executive::_executiveMapMtx; ///< protects _executiveMap -map> Executive::_executiveMap; ///< Map of executives for queries in -progress. -*/ - //////////////////////////////////////////////////////////////////////// // class Executive implementation //////////////////////////////////////////////////////////////////////// @@ -124,7 +118,7 @@ Executive::Executive(ExecutiveConfig const& c, shared_ptr c } Executive::~Executive() { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::~Executive() " << getIdStr()); + LOGS(_log, LOG_LVL_DEBUG, "Executive::~Executive() " << getIdStr()); qdisp::CzarStats::get()->deleteQuery(); qdisp::CzarStats::get()->deleteJobs(_incompleteJobs.size()); // Remove this executive from the map. @@ -137,7 +131,6 @@ Executive::~Executive() { _asyncTimer->cancel(); qdisp::CzarStats::get()->untrackQueryProgress(_id); } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::~Executive() " << getIdStr() << " end"); } Executive::Ptr Executive::create(ExecutiveConfig const& c, shared_ptr const& ms, @@ -193,31 +186,10 @@ void Executive::setQueryId(QueryId id) { _idStr = QueryIdHelper::makeIdStr(_id); // Insert into the global executive map. - { - /* &&& - lock_guard lgMap(_executiveMapMtx); - _executiveMap[_id] = shared_from_this(); - */ - czar::Czar::getCzar()->insertExecutive(_id, shared_from_this()); - } + { czar::Czar::getCzar()->insertExecutive(_id, shared_from_this()); } qdisp::CzarStats::get()->trackQueryProgress(_id); } -/* &&& -Executive::Ptr Executive::getExecutiveFromMap(QueryId qId) { - lock_guard lgMap(_executiveMapMtx); - auto iter = _executiveMap.find(qId); - if (iter == _executiveMap.end()) { - return nullptr; - } - Executive::Ptr exec = iter->second.lock(); - if (exec == nullptr) { - _executiveMap.erase(iter); - } - return exec; -} -*/ - UberJob::Ptr Executive::findUberJob(UberJobId ujId) { lock_guard lgMap(_uberJobsMapMtx); auto iter = _uberJobsMap.find(ujId); @@ -230,7 +202,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. @@ -259,7 +230,7 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { return jobQuery; } - _addToChunkJobMap(jobQuery); // &&&uj + _addToChunkJobMap(jobQuery); } if (_empty.exchange(false)) { @@ -270,50 +241,38 @@ JobQuery::Ptr Executive::add(JobDescription::Ptr const& jobDesc) { QSERV_LOGCONTEXT_QUERY_JOB(jobQuery->getQueryId(), jobQuery->getJobId()); - LOGS(_log, LOG_LVL_WARN, "&&& Executive::add end"); return jobQuery; } void Executive::runJobQuery(JobQuery::Ptr const& jobQuery) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::runJobQuery start"); bool started = jobQuery->runJob(); if (!started && isLimitRowComplete()) { markCompleted(jobQuery->getJobId(), false); } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::runJobQuery end"); } void Executive::queueJobStart(PriorityCommand::Ptr const& cmd) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueJobStart start"); _jobStartCmdList.push_back(cmd); if (_scanInteractive) { _qdispPool->queCmd(cmd, 0); } else { _qdispPool->queCmd(cmd, 1); } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueJobStart end"); } void Executive::queueFileCollect(PriorityCommand::Ptr const& cmd) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect start"); - if (_scanInteractive) { _qdispPool->queCmd(cmd, 3); } else { _qdispPool->queCmd(cmd, 4); } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::queueFileCollect end"); } void Executive::runUberJob(std::shared_ptr const& uberJob) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::runUberJob start"); - - bool const useqdisppool = true; /// &&& delete + /// TODO:UJ delete useqdisppool, only set to false if problems during testing + bool const useqdisppool = true; if (useqdisppool) { - auto runUberJobFunc = [uberJob](util::CmdData*) { - LOGS(_log, LOG_LVL_WARN, "&&&uj Executive::runUberJob::runUberJobFunc a"); - uberJob->runUberJob(); - }; + auto runUberJobFunc = [uberJob](util::CmdData*) { uberJob->runUberJob(); }; auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(runUberJobFunc)); _jobStartCmdList.push_back(cmd); @@ -325,12 +284,10 @@ void Executive::runUberJob(std::shared_ptr const& uberJob) { } 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(); @@ -340,19 +297,15 @@ void Executive::waitForAllJobsToStart() { cmd->waitComplete(); } LOGS(_log, LOG_LVL_INFO, "waitForAllJobsToStart done"); - LOGS(_log, LOG_LVL_WARN, "&&& waitForAllJobsToStart end"); } // 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) -// -bool Executive::startQuery(shared_ptr const& jobQuery) { // &&& - LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery start"); +// // TODO:UJ delete this function +bool Executive::startQuery(shared_ptr const& jobQuery) { lock_guard lock(_cancelled.getMutex()); - LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery a"); - // 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(). @@ -371,13 +324,11 @@ bool Executive::startQuery(shared_ptr const& jobQuery) { // &&& // Start the query. The rest is magically done in the background. // - // &&&uj sending mechanism needs to change to be like AddReplicaQservMgtRequest::createHttpReqImp getXrdSsiService()->ProcessRequest(*(qr.get()), jobResource); - LOGS(_log, LOG_LVL_WARN, "&&& Executive::startQuery end"); return true; } -Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { // &&& +Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { lock_guard lck(_chunkToJobMapMtx); ChunkIdJobMapType unassignedMap; @@ -389,7 +340,7 @@ Executive::ChunkIdJobMapType Executive::unassignedChunksInQuery() { // &&& return unassignedMap; } -void Executive::addUberJobs(std::vector> const& uJobsToAdd) { // &&& +void Executive::addUberJobs(std::vector> const& uJobsToAdd) { lock_guard lck(_uberJobsMapMtx); for (auto const& uJob : uJobsToAdd) { UberJobId ujId = uJob->getJobId(); @@ -423,32 +374,6 @@ void Executive::assignJobsToUberJobs() { } } -bool Executive::startUberJob(UberJob::Ptr const& uJob) { // &&& - - lock_guard lock(_cancelled.getMutex()); - - // If this has been cancelled, then return false. - // - if (_cancelled) return false; - - // &&&uj NEED CODE to put call to runUberJob into the priority queue. - - return true; -} - -JobQuery::Ptr Executive::getSharedPtrForRawJobPtr(JobQuery* jqRaw) { //&&& - assert(jqRaw != nullptr); - int jobId = jqRaw->getJobId(); - 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; -} - -/// &&& doc void Executive::addMultiError(int errorCode, std::string const& errorMsg, int errorState) { util::Error err(errorCode, errorMsg, errorState); { @@ -479,7 +404,6 @@ bool Executive::join() { static bool func(Executive::JobMap::value_type const& entry) { qmeta::JobStatus::Info const& esI = entry.second->getStatus()->getInfo(); LOGS(_log, LOG_LVL_TRACE, "entry state:" << (void*)entry.second.get() << " " << esI); - LOGS(_log, LOG_LVL_WARN, "&&& entry state:" << (void*)entry.second.get() << " " << esI); return (esI.state == qmeta::JobStatus::RESPONSE_DONE) || (esI.state == qmeta::JobStatus::COMPLETE); } @@ -561,7 +485,6 @@ void Executive::markCompleted(JobId jobId, bool success) { } void Executive::squash() { - LOGS(_log, LOG_LVL_WARN, "&&&uj Executive::squash() qid=" << getId() << " str=" << getIdStr()); bool alreadyCancelled = _cancelled.exchange(true); if (alreadyCancelled) { LOGS(_log, LOG_LVL_DEBUG, "Executive::squash() already cancelled! refusing. qid=" << getId()); @@ -583,6 +506,8 @@ void Executive::squash() { // 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"); @@ -619,9 +544,13 @@ void Executive::_squashSuperfluous() { } 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, - "&&& NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId + " - "queryId."); + "TODO:UJ NEED CODE Executive::sendWorkerCancelMsg to send messages to workers to cancel this czarId " + "+ " + "queryId. " + << deleteResults); } int Executive::getNumInflight() const { @@ -824,14 +753,10 @@ void Executive::_waitAllUntilEmpty() { void Executive::_addToChunkJobMap(JobQuery::Ptr const& job) { int chunkId = job->getDescription()->resource().chunk(); auto entry = pair(chunkId, job); - // LOGS(_log, LOG_LVL_WARN, "&&& _addToChunkJobMap chunkId=" << chunkId); &&& lock_guard lck(_chunkToJobMapMtx); - if (_chunkToJobMapInvalid) { - throw util::Bug(ERR_LOC, "&&& map insert FAILED, map is already invalid"); - } bool inserted = _chunkToJobMap.insert(entry).second; if (!inserted) { - throw util::Bug(ERR_LOC, "&&& map insert FAILED ChunkId=" + to_string(chunkId) + " already existed"); + throw util::Bug(ERR_LOC, "map insert FAILED ChunkId=" + to_string(chunkId) + " already existed"); } } @@ -883,194 +808,4 @@ void Executive::_printState(ostream& os) { } } -#if 0 //&&& -void Executive::buildAndSendUberJobs(int const maxChunksPerUber) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs a"); - vector uberJobs; - - auto czarPtr = czar::Czar::getCzar(); - auto czChunkMap = czarPtr->getCzarChunkMap(); - auto czRegistry = czarPtr->getCzarRegistry(); - - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs b"); - auto const [chunkMapPtr, workerChunkMapPtr] = czChunkMap->getMaps(); //&&&uj - - // Make a map of all jobs in the executive. - // &&& TODO:UJ for now, just using ints. At some point, need to check that ResourceUnit databases can - // be found for all databases in the query - /* &&& instead of destroying the only copy, going to make a copy that only contains - * unassigned jobs. - qdisp::Executive::ChunkIdJobMapType chunksInQuery = _executive->getChunkJobMapAndInvalidate(); - if (_chunkToJobMapInvalid.exchange(true)) { - throw util::Bug(ERR_LOC, "getChunkJobMapInvalidate called when map already invalid"); - } - return _chunkToJobMap; - - */ - ChunkIdJobMapType unassignedChunksInQuery; - { - lock_guard lckChunkJobMap(_chunkToJobMapMtx); - for(auto const& [chunkId, jobPtr] : _chunkToJobMap) { - if (!jobPtr->isInUberJob()) { - unassignedChunksInQuery[chunkId] = jobPtr; - } - } - } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs c"); - - // &&& TODO:UJ skipping in proof of concept: get a list of all databases in jobs (all jobs should use - // the same databases) Use this to check for conflicts - - /* &&& - // assign jobs to uberJobs - int maxChunksPerUber = 3; // &&&uj maybe put in config??? or set on command line?? - // &&&uj Different queries may benefit from different values - // &&&uj Such as LIMIT=1 may work best with this at 1, where - // &&&uj 100 would be better for others. - */ - // keep cycling through workers until no more chunks to place. - - // TODO:UJ &&&uj Once everything is an UberJob, it can start with 1 or 0. - //int _uberJobId = qdisp::UberJob::getFirstIdNumber(); - - // &&&uj - // - 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. - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d"); - for (auto const& [chunkId, jqPtr] : unassignedChunksInQuery) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d1"); - auto iter = chunkMapPtr->find(chunkId); - if (iter == chunkMapPtr->end()) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d1a"); - missingChunks.push_back(chunkId); - break; - } - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d2");; - czar::CzarChunkMap::ChunkData::Ptr chunkData = iter->second; - auto targetWorker = chunkData->getPrimaryScanWorker().lock(); - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d3"); - if (targetWorker == nullptr) { //&&&uj if (targetWorker == nullptr || this worker already tried for this chunk) { - LOGS(_log, LOG_LVL_ERROR, "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, "Alternate worker found for chunk=" << chunkData->dump()); - } - } - if (!found) { - // &&&uj If enough workers are down, there will be a chunk that cannot be found. - // the correct course of action is probably to check the Registry, and - // after so many attempts, cancel the user query with a - // "chunk(s)[list of missing chunks]" error. Perhaps, the attemptCount - // in the Job or JobDescription could be used for this. - LOGS(_log, LOG_LVL_ERROR, - "No primary or alternate worker found for chunk=" << chunkData->dump()); - throw util::Bug(ERR_LOC, string("No primary or alternate worker found for chunk.") + - " Crashing the program here for this reason is not " - "appropriate. &&& NEEDS CODE"); - } - } - // Add this job to the appropriate UberJob, making the UberJob if needed. - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d4"); - string workerId = targetWorker->getWorkerId(); - auto& ujVect = workerJobMap[workerId]; - if (ujVect.empty() || ujVect.back()->getJobCount() > maxChunksPerUber) { - LOGS(_log, LOG_LVL_WARN, "&&& Executive::buildandSendUberJobs d4a"); - auto ujId = _uberJobId++; // keep ujId consistent - string uberResultName = _ttn->make(ujId); - auto respHandler = make_shared(_infileMerger, uberResultName); - auto thisExec = shared_from_this(); - auto uJob = qdisp::UberJob::create(thisExec, respHandler, getId(), ujId, - _qMetaCzarId, targetWorker); - ujVect.push_back(uJob); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c6"); - ujVect.back()->addJob(jqPtr); - } - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c7"); - if (!missingChunks.empty()) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e1c8"); - string errStr = string(__func__) + " a worker could not be found for these chunks "; - for (auto const& chk : missingChunks) { - errStr += to_string(chk) + ","; - } - LOGS(_log, LOG_LVL_ERROR, errStr); - throw util::Bug( - ERR_LOC, - errStr + " Crashing the program here for this reason is not appropriate. &&& NEEDS CODE"); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew e end"); - - //&&&uj - // Add worker contact info to UberJobs. - auto const wContactMap = czRegistry->getWorkerContactMap(); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f"); - for (auto const& [wIdKey, ujVect] : workerJobMap) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f1"); - auto iter = wContactMap->find(wIdKey); - if (iter == wContactMap->end()) { - // &&&uj Not appropriate to throw for this. Need to re-direct all jobs to different workers. - throw util::Bug(ERR_LOC, string(" &&&uj NEED CODE, no contact information for ") + wIdKey); - } - auto const& wContactInfo = iter->second; - for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f2"); - ujPtr->setWorkerContactInfo(wContactInfo); - } - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f3"); - _executive->addUberJobs(ujVect); - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f4"); - for (auto const& ujPtr : ujVect) { - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew f5"); - //&&&uj IMPORTANT - //&&&uj This just sends the test message to the worker, but the - // worker only parses it and sends a message back. The - // worker does not create and run tasks at this point. - // The call to runUberJob here should be replaced by a call - // to startUberJob that puts the call to runUberJob into - // the a priority queue command. - _executive->runUberJob(ujPtr); - } - } - - LOGS(_log, LOG_LVL_WARN, "&&& UserQuerySelect::submitNew g"); //&&&uj - - LOGS(_log, LOG_LVL_DEBUG, "total jobs in query=" << sequence); - _executive->waitForAllJobsToStart(); - - // we only care about per-chunk info for ASYNC queries - if (_async) { - std::lock_guard lock(chunksMtx); - _qMetaAddChunks(chunks); - } -} -#endif //&&& - } // namespace lsst::qserv::qdisp diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 3930d7df80..1d95e5a9ca 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -51,8 +51,8 @@ #include "util/threadSafe.h" #include "util/ThreadPool.h" -//&&& replace with better enable/disable feature. -#define uberJobsEnabled 1 //&&&uj || true +// TODO:UJ replace with better enable/disable feature, or just use only UberJobs +#define uberJobsEnabled 1 // Forward declarations class XrdSsiService; @@ -74,7 +74,7 @@ class QuerySession; namespace qdisp { class JobQuery; -class UberJob; //&&&uj +class UberJob; } // namespace qdisp namespace rproc { @@ -103,7 +103,7 @@ class Executive : public std::enable_shared_from_this { public: typedef std::shared_ptr Ptr; typedef std::unordered_map> JobMap; - typedef int ChunkIdType; //&&&uj This type is probably not needed + typedef int ChunkIdType; typedef std::map> ChunkIdJobMapType; /// Construct an Executive. @@ -117,36 +117,31 @@ class Executive : public std::enable_shared_from_this { ~Executive(); - /* &&& - /// &&& doc - static Ptr getExecutiveFromMap(QueryId qId); - */ - std::string cName(const char* funcName = "") { return std::string("Executive::") + funcName; } - /// &&&uj doc + /// 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; } - //&&&void buildAndSendUberJobs(int const maxChunksPerUber); - /// &&&uj doc Return a map that only contains Jobs not assigned to an UberJob. + /// Return a map that only contains Jobs not assigned to an UberJob. ChunkIdJobMapType unassignedChunksInQuery(); - /// &&& doc + /// 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); - /// &&& doc - to be deleted + /// TODO:UJ - to be deleted void runJobQuery(std::shared_ptr const& jobQuery); - // &&&uj doc + // 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); - /// &&& doc + /// 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 @@ -169,10 +164,6 @@ class Executive : public std::enable_shared_from_this { /// QMeta, which isn't called until some basic checks on the user query /// have passed. void setQueryId(QueryId id); - //&&&void setTmpTableNameGenerator(std::shared_ptr const& ttn) { _ttn = ttn; } - - //&&&void setInfileMerger(std::shared_ptr infileMerger) { _infileMerger = - // infileMerger; } QueryId getId() const { return _id; } std::string const& getIdStr() const { return _idStr; } @@ -192,7 +183,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); @@ -214,21 +205,23 @@ class Executive : public std::enable_shared_from_this { /// @see python module lsst.qserv.czar.proxy.unlock() void updateProxyMessages(); - /// Add UbjerJobs to this user query. &&& + /// Add UbjerJobs to this user query. void addUberJobs(std::vector> const& jobsToAdd); - /// &&&uj doc + /// Call UserQuerySelect::buildAndSendUberJobs make new UberJobs for + /// unassigned jobs. void assignJobsToUberJobs(); - //&&& ChunkIdJobMapType& getChunkJobMapAndInvalidate(); /// &&& delete - bool startUberJob(std::shared_ptr const& uJob); /// &&& - std::shared_ptr getSharedPtrForRawJobPtr(JobQuery* jqRaw); /// &&& delete - 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; } - /// &&& doc + /// 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; @@ -305,7 +298,10 @@ class Executive : public std::enable_shared_from_this { mutable std::mutex _errorsMutex; std::condition_variable _allJobsComplete; - mutable std::recursive_mutex _jobMapMtx; // &&& see what it takes to make this a normal mutex + // 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. std::string _idStr{QueryIdHelper::makeIdStr(0, true)}; @@ -319,18 +315,17 @@ class Executive : public std::enable_shared_from_this { bool _scanInteractive = false; ///< true for interactive scans. - // Add a job to the _chunkToJobMap //&&&uj - void _addToChunkJobMap(std::shared_ptr const& job); //&&&uj - /// _chunkToJobMap is created once and then destroyed after use. - std::atomic _chunkToJobMapInvalid{ - false}; ///< true indicates the map is no longer valid. //&&&uj delete - std::mutex _chunkToJobMapMtx; ///< protects _chunkToJobMap //&&&uj - ChunkIdJobMapType _chunkToJobMap; ///< Map of jobs ordered by chunkId //&&&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. //&&&uj + 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. @@ -349,20 +344,18 @@ class Executive : public std::enable_shared_from_this { std::atomic _queryIdSet{false}; ///< Set to true when _id is set. - ///&&&uj + /// Weak pointer to the UserQuerySelect object for this query. std::weak_ptr _userQuerySelect; - /// If true, there are probably jobs that need to be reassigned to new - /// UberJobs. - /// &&&uj NEED CODE - at some point this needs to be checked so the - /// executive can make new uberjobs. + /// 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}; }; -/// &&&uj MarkCompleteFunc is not needed with uberjobs. +/// TODO:UJ delete - MarkCompleteFunc is not needed with uberjobs. class MarkCompleteFunc { public: typedef std::shared_ptr Ptr; diff --git a/src/qdisp/JobDescription.cc b/src/qdisp/JobDescription.cc index de8db60445..50c05c39e6 100644 --- a/src/qdisp/JobDescription.cc +++ b/src/qdisp/JobDescription.cc @@ -65,7 +65,7 @@ JobDescription::JobDescription(qmeta::CzarId czarId, QueryId qId, JobId jobId, R _chunkResultName(chunkResultName), _mock(mock) {} -bool JobDescription::incrAttemptCountScrubResults() { // &&& to be deleted +bool JobDescription::incrAttemptCountScrubResults() { // TODO:UJ delete if (_attemptCount >= 0) { _respHandler->prepScrubResults(_jobId, _attemptCount); // Registers the job-attempt as invalid } @@ -105,7 +105,7 @@ bool JobDescription::incrAttemptCountScrubResultsJson(std::shared_ptr // build the request auto js = _taskMsgFactory->makeMsgJson(*_chunkQuerySpec, _chunkResultName, _queryId, _jobId, _attemptCount, _czarId); - LOGS(_log, LOG_LVL_ERROR, "&&& JobDescription::incrAttemptCountScrubResultsJson js=" << (*js)); + LOGS(_log, LOG_LVL_DEBUG, "JobDescription::" << __func__ << " js=" << (*js)); _jsForWorker = js; return true; @@ -118,16 +118,7 @@ void JobDescription::buildPayload() { _payloads[_attemptCount] = os.str(); } -/* &&& -bool JobDescription::fillTaskMsg(proto::TaskMsg* tMsg) { //&&&uj -probably just delete. - //&&&uj return _taskMsgFactory->fillTaskMsg(tMsg, *_chunkQuerySpec, _chunkResultName, _queryId, - //_jobId, _attemptCount, _czarId); - util::Bug(ERR_LOC, "&&& JobDescription::fillTaskMsg"); - return false; -} -*/ - -bool JobDescription::verifyPayload() const { //&&&uj - is there any value to this now? +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 88aa48c3b2..8c61f9d656 100644 --- a/src/qdisp/JobDescription.h +++ b/src/qdisp/JobDescription.h @@ -90,16 +90,15 @@ class JobDescription { /// @returns true when _attemptCount is incremented correctly and the payload is built. /// If the starting value of _attemptCount was greater than or equal to zero, that /// attempt is scrubbed from the result table. - bool incrAttemptCountScrubResults(); // &&&uj - to be deleted - /// doc &&&uj - scrubbing results probably unneeded with uj. This should be renamed. + bool incrAttemptCountScrubResults(); // TODO:UJ - to be deleted + /// Increase the attempt count by 1 and return false if that puts it over the limit. + /// TODO:UJ scrubbing results unneeded with uj. This should be renamed. bool incrAttemptCountScrubResultsJson(std::shared_ptr const& exec, bool increase); bool verifyPayload() const; ///< @return true if the payload is acceptable to protobufs. - //&&&bool fillTaskMsg(proto::TaskMsg* tMsg); //&&&uj - std::shared_ptr getJsForWorker() { return _jsForWorker; } - void resetJsForWorker() { _jsForWorker.reset(); } // &&&uj may need mutex for _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 5ebc7c6d26..9b99f4d9da 100644 --- a/src/qdisp/JobQuery.cc +++ b/src/qdisp/JobQuery.cc @@ -66,7 +66,7 @@ JobQuery::~JobQuery() { /** 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() { // &&& +bool JobQuery::runJob() { // TODO:UJ delete QSERV_LOGCONTEXT_QUERY_JOB(getQueryId(), getJobId()); LOGS(_log, LOG_LVL_DEBUG, " runJob " << *this); auto executive = _executive.lock(); diff --git a/src/qdisp/JobQuery.h b/src/qdisp/JobQuery.h index 3cc1d9ee21..a11b628d49 100644 --- a/src/qdisp/JobQuery.h +++ b/src/qdisp/JobQuery.h @@ -44,7 +44,7 @@ class QdispPool; class QueryRequest; /// This class is used to describe, monitor, and control a single query to a worker. -/// &&&uj once all Jobs are sent out as UberJobs, the purpose of this class is a bit +/// 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 { @@ -62,7 +62,7 @@ class JobQuery : public JobBase { virtual ~JobQuery(); - /// &&& doc + /// Run this job. bool runJob(); QueryId getQueryId() const override { return _qid; } @@ -129,17 +129,6 @@ class JobQuery : public JobBase { _jobDescription->respHandler()->setJobQuery(jbPtr); } - /* &&& - /// NOTE: _rmutex must be held before calling this - int _getRunAttemptsCount() const { - std::lock_guard lock(_rmutex); - return _jobDescription->getAttemptCount(); - } - - int _getMaxAttempts() const { return 5; } // Arbitrary value until solid value with reason determined. - int _getAttemptSleepSeconds() const { return 15; } // As above or until added to config file. - */ - /// @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 @@ -164,7 +153,7 @@ class JobQuery : public JobBase { std::string const _idStr; ///< Identifier string for logging. // Values that need mutex protection - // &&&uj recursive can probably go away with as well as _inSsi. + // TODO:UJ recursive can probably go away with as well as _inSsi. mutable std::recursive_mutex _rmutex; ///< protects _jobDescription, ///< _queryRequestPtr, _uberJobId, ///< and _inSsi diff --git a/src/qdisp/QueryRequest.cc b/src/qdisp/QueryRequest.cc index 7da1046846..2821f7fbae 100644 --- a/src/qdisp/QueryRequest.cc +++ b/src/qdisp/QueryRequest.cc @@ -317,16 +317,17 @@ void QueryRequest::cleanup() { /// 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. +// TODO:UJ Delete QueryRequest class, including this function. bool QueryRequest::_errorFinish(bool shouldCancel) { LOGS(_log, LOG_LVL_DEBUG, "_errorFinish() shouldCancel=" << shouldCancel); auto jbase = _job; JobQuery::Ptr jq = dynamic_pointer_cast(jbase); if (jq == nullptr) { - //&&&uj IMPORTANT UberJob failures are different than JobQuery failures. + // TODO:UJ The QueryRequest class will be deleted, so this doen't matter. UberJob::Ptr uberJob = dynamic_pointer_cast(jbase); if (uberJob != nullptr) { - throw util::Bug(ERR_LOC, "&&&NEED_CODE for _errorFinish to work correctly with UberJob"); + throw util::Bug(ERR_LOC, " for _errorFinish to work correctly with UberJob"); // UberJobs breakup into their JobQueries when they fail and run the jobs directly. } return false; @@ -402,17 +403,6 @@ void QueryRequest::_finish() { 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); - } -} -*/ - void QueryRequest::_callMarkComplete(bool success) { if (!_calledMarkComplete.exchange(true)) { auto jq = _job; diff --git a/src/qdisp/QueryRequest.h b/src/qdisp/QueryRequest.h index 74826d16cd..1327b4673e 100644 --- a/src/qdisp/QueryRequest.h +++ b/src/qdisp/QueryRequest.h @@ -79,16 +79,11 @@ class RequestError : public std::exception { /// 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. +// TODO:UJ delete this class class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this { public: typedef std::shared_ptr Ptr; - /* &&& - static Ptr create(std::shared_ptr const& jobQuery) { - Ptr newQueryRequest(new QueryRequest(jobQuery)); - return newQueryRequest; - } - */ static Ptr create(std::shared_ptr const& jobBase) { Ptr newQueryRequest(new QueryRequest(jobBase)); return newQueryRequest; @@ -123,26 +118,17 @@ class QueryRequest : public XrdSsiRequest, public std::enable_shared_from_this const& jobQuery); QueryRequest(JobBase::Ptr const& job); /// Inform the Executive that this query completed, and call MarkCompleteFunc only once. /// This should only be called from _finish() or _errorFinish. void _callMarkComplete(bool success); - //&&&bool _importResultFile(JobQuery::Ptr const& jq); bool _importResultFile(JobBase::Ptr const& jq); bool _importError(std::string const& msg, int code); bool _errorFinish(bool stopTrying = false); void _finish(); - //&&&void _flushError(JobQuery::Ptr const& jq); void _flushError(JobBase::Ptr const& jq); - /* &&& - /// 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; - */ /// Job information. Not using a weak_ptr as Executive could drop its JobBase::Ptr before we're done with /// it. A call to cancel() could reset _job early, so copy or protect _job with _finishStatusMutex as /// needed. If (_finishStatus == ACTIVE) _job should be good. diff --git a/src/qdisp/ResponseHandler.h b/src/qdisp/ResponseHandler.h index 5f2cf1d308..ec157d8a76 100644 --- a/src/qdisp/ResponseHandler.h +++ b/src/qdisp/ResponseHandler.h @@ -66,7 +66,7 @@ class ResponseHandler { /// @return true if successful (no error) virtual bool flush(proto::ResponseSummary const& responseSummary, uint32_t& resultRows) = 0; - /// &&&uj doc + /// 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 @@ -74,7 +74,7 @@ class ResponseHandler { virtual std::tuple flushHttp(std::string const& fileUrl, uint64_t expectedRows, uint64_t& resultRows) = 0; - /// &&& doc + /// 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. diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 5aae13d330..8b092a6d3b 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -89,37 +89,27 @@ bool UberJob::addJob(JobQuery::Ptr const& job) { success = true; } if (!success) { - // &&&uj not really the right thing to do, but high visibility wanted for now. + // 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_WARN, cName(__func__) << "&&& UberJob::runUberJob() start"); - // &&&uj most, if not all, of this should be done in a command in the QDispPool. - // &&&uk especially the communication parts. - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() a"); + LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); // Build the uberjob payload for each job. nlohmann::json uj; unique_lock jobsLock(_jobsMtx); - LOGS(_log, LOG_LVL_WARN, - getIdStr() << "&&&uj count qid=" << getQueryId() << " ujId=" << getJobId() - << " jobs.sz=" << _jobs.size()); auto exec = _executive.lock(); for (auto const& jqPtr : _jobs) { - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() a1 " << jqPtr->getIdStr()); jqPtr->getDescription()->incrAttemptCountScrubResultsJson(exec, true); } - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() b"); // Send the uberjob to the worker auto const method = http::Method::POST; string const url = "http://" + _wContactInfo->wHost + ":" + to_string(_wContactInfo->wPort) + "/queryjob"; - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() c " << url); vector const headers = {"Content-Type: application/json"}; auto const& czarConfig = cconfig::CzarConfig::instance(); - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() c"); // See xrdsvc::httpWorkerCzarModule::_handleQueryJob for json message parsing. json request = {{"version", http::MetaModule::version}, {"instance_id", czarConfig->replicationInstanceId()}, @@ -136,33 +126,24 @@ bool UberJob::runUberJob() { {"czarid", _czarId}, {"jobs", json::array()}}}}; - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() d " << request); auto& jsUberJob = request["uberjob"]; - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() e " << jsUberJob); auto& jsJobs = jsUberJob["jobs"]; - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob::runUberJob() f " << jsJobs); for (auto const& jbPtr : _jobs) { - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1"); auto const description = jbPtr->getDescription(); - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1a"); if (description == nullptr) { - throw util::Bug(ERR_LOC, - cName(__func__) + " &&&uj description=null for job=" + jbPtr->getIdStr()); + throw util::Bug(ERR_LOC, cName(__func__) + " description=null for job=" + jbPtr->getIdStr()); } auto const jsForWorker = jbPtr->getDescription()->getJsForWorker(); - LOGS(_log, LOG_LVL_WARN, getIdStr() << "&&& UberJob ::runUberJob() f1b"); if (jsForWorker == nullptr) { - throw util::Bug(ERR_LOC, getIdStr() + " &&&uj jsForWorker=null for job=" + jbPtr->getIdStr()); + throw util::Bug(ERR_LOC, cName(__func__) + " jsForWorker=null for job=" + jbPtr->getIdStr()); } - //&&& json jsJob = {{"jobdesc", *(jbPtr->getDescription()->getJsForWorker())}}; 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_WARN, getIdStr() << "&&& UberJob ::runUberJob() g"); - LOGS(_log, LOG_LVL_WARN, __func__ << " &&&REQ " << request); + 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() @@ -172,37 +153,33 @@ bool UberJob::runUberJob() { string exceptionWhat; try { json const response = client.readAsJson(); - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() response=" << response); if (0 != response.at("success").get()) { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() success"); transmitSuccess = true; } else { - LOGS(_log, LOG_LVL_WARN, _idStr << " UberJob::" << __func__ << " response success=0"); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " response success=0"); } } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); + LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); exceptionWhat = ex.what(); } if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, "&&&uj UberJob::runUberJob() transmit failure, try to send jobs elsewhere"); 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 { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::runUberJob() register all jobs as transmitted to worker"); setStatusIfOk(qmeta::JobStatus::REQUEST, cName(__func__) + " transmitSuccess"); // locks _jobsMtx } return false; } void UberJob::prepScrubResults() { - // &&&uj There's a good chance this will not be needed as incomplete files will not be merged - // so you don't have to worry about removing rows from incomplete jobs or uberjobs - // from the result table. + // 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, - "&&&uj If needed, should call prepScrubResults for all JobQueries in the UberJob "); + "TODO:UJ If needed, should call prepScrubResults for all JobQueries in the UberJob "); } void UberJob::_unassignJobs() { @@ -212,7 +189,6 @@ void UberJob::_unassignJobs() { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " exec is null"); return; } - //&&&auto maxAttempts = exec->getMaxAttempts(); for (auto&& job : _jobs) { string jid = job->getIdStr(); if (!job->unassignFromUberJob(getJobId())) { @@ -222,14 +198,6 @@ void UberJob::_unassignJobs() { exec->squash(); return; } - /* &&& - auto attempts = job->getAttemptCount(); - if (attempts > maxAttempts) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " job=" << jid << " attempts=" << attempts << " - maxAttempts reached, cancelling"); exec->addMultiError(qmeta::JobStatus::RETRY_ERROR, "max attempts - reached " + to_string(attempts) + " job=" + jid, util::ErrorCode::INTERNAL); exec->squash(); return; - } - */ LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " job=" << jid << " attempts=" << job->getAttemptCount()); } @@ -247,15 +215,6 @@ bool UberJob::isQueryCancelled() { return exec->getCancelled(); } -bool UberJob::verifyPayload() const { - proto::ProtoImporter pi; - if (!pi.messageAcceptable(_payload)) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " Error serializing UberJobMsg."); - return false; - } - return true; -} - bool UberJob::_setStatusIfOk(qmeta::JobStatus::State newState, string const& msg) { // must be locked _jobsMtx auto currentState = _jobStatus->getState(); @@ -289,7 +248,6 @@ void UberJob::callMarkCompleteFunc(bool 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. - // &&&uj JobStatus should have a separate entry for success/failure/incomplete/retry. string source = string("UberJob_") + (success ? "SUCCESS" : "FAILED"); _jobStatus->updateInfo(getIdStr(), qmeta::JobStatus::COMPLETE, source); for (auto&& job : _jobs) { @@ -311,25 +269,19 @@ void UberJob::callMarkCompleteFunc(bool success) { /// 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_WARN, "&&&uj UberJob::importResultFile a"); - LOGS(_log, LOG_LVL_WARN, - cName(__func__) << "&&&uj fileUrl=" << fileUrl << " rowCount=" << rowCount - << " fileSize=" << fileSize); LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " fileUrl=" << fileUrl << " rowCount=" << rowCount << " fileSize=" << fileSize); if (isQueryCancelled()) { - LOGS(_log, LOG_LVL_WARN, "UberJob::importResultFile import job was cancelled."); + LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); return _importResultError(true, "cancelled", "Query cancelled"); } - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile b"); 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"); } - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile c"); if (exec->isLimitRowComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); @@ -341,24 +293,20 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return _importResultError(false, "rowLimited", "Enough rows already"); } - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile d"); - 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__) << " &&&uj setStatusFail could not set status to RESPONSE_READY"); + 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); - // &&&uj lambda may not be the best way to do this. - // &&&uj check synchronization - may need a mutex for merging. + // 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*) { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc a"); auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { LOGS(_log, LOG_LVL_DEBUG, @@ -368,7 +316,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ uint64_t resultRows = 0; auto [flushSuccess, flushShouldCancel] = ujPtr->getRespHandler()->flushHttp(fileUrl, rowCount, resultRows); - LOGS(_log, LOG_LVL_WARN, ujPtr->cName(__func__) << "::fileCollectFunc &&&uj b"); + LOGS(_log, LOG_LVL_DEBUG, ujPtr->cName(__func__) << "::fileCollectFunc"); if (!flushSuccess) { // This would probably indicate malformed file+rowCount or // writing the result table failed. @@ -377,10 +325,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ // At this point all data for this job have been read, there's no point in // having XrdSsi wait for anything. - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc c"); ujPtr->_importResultFinish(resultRows); - - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::importResultFile::fileCollectFunc end"); }; auto cmd = qdisp::PriorityCommand::Ptr(new qdisp::PriorityCommand(fileCollectFunc)); @@ -394,8 +339,6 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ } json UberJob::workerError(int errorCode, string const& errorMsg) { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError a"); - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError code=" << errorCode << " msg=" << errorMsg); LOGS(_log, LOG_LVL_WARN, cName(__func__) << " errcode=" << errorCode << " errmsg=" << errorMsg); bool const deleteData = true; @@ -405,7 +348,6 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " no executive or cancelled"); return _workerErrorFinish(deleteData, "cancelled"); } - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError c"); if (exec->isLimitRowComplete()) { int dataIgnored = exec->incrDataIgnoredCount(); @@ -417,8 +359,6 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { return _workerErrorFinish(keepData, "none", "limitRowComplete"); } - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::workerError d"); - // 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 @@ -426,7 +366,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; //&&& delete after testing &&&&&&& + 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) @@ -445,8 +385,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { 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. + // In all cases, the worker should delete the file as this czar will not ask for it. auto exec = _executive.lock(); if (exec != nullptr) { @@ -476,8 +415,7 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str } nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish a"); - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << "&&&uj start"); // &&& keep + 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 @@ -500,18 +438,16 @@ nlohmann::json UberJob::_importResultFinish(uint64_t resultRows) { exec->checkLimitRowComplete(); json jsRet = {{"success", 1}, {"errortype", ""}, {"note", ""}}; - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish end"); return jsRet; } nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, std::string const& note) { - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_workerErrorFinish a"); - /// 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??? - /// &&& Only recoverable errors would be: communication failure, or missing table ??? - /// Return a "success:1" json message to be sent to the worker. + // 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"); @@ -519,7 +455,6 @@ nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& e } json jsRet = {{"success", 1}, {"deletedata", deleteData}, {"errortype", ""}, {"note", ""}}; - LOGS(_log, LOG_LVL_WARN, "&&&uj UberJob::_importResultFinish end"); return jsRet; } diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index ad25e645a1..0dd2f69cdc 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -25,8 +25,8 @@ // Qserv headers #include "qmeta/types.h" -#include "czar/CzarChunkMap.h" // Need nested class. &&&uj Make non-nested? -#include "czar/CzarRegistry.h" // Need nested class. &&&uj Make non-nested? +#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" @@ -37,10 +37,14 @@ class JobQuery; class QueryRequest; -/// &&& doc /// 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; @@ -55,18 +59,18 @@ class UberJob : public JobBase { virtual ~UberJob(){}; - static int getFirstIdNumber() { return 9'000'000; } // &&&uj this can probably be 0 now. - 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; } // TODO:UJ relocate to JobBase - UberJobId getJobId() const override { return _uberJobId; } // &&&uj change name + 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; } + 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; @@ -77,7 +81,7 @@ class UberJob : public JobBase { 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. &&&uj + ; // 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 @@ -90,25 +94,24 @@ class UberJob : public JobBase { return _setStatusIfOk(newState, msg); } - bool verifyPayload() const; - int getJobCount() const { return _jobs.size(); } - /// &&&uj uj may not need, + /// TODO:UJ may not need, void prepScrubResults(); - //&&&uj + /// 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; } - //&&&uj + /// Get the data for the worker that should handle this UberJob. czar::CzarChunkMap::WorkerChunksData::Ptr getWorkerData() { return _workerData; } - /// &&&uj doc + /// Collect and merge the results from the worker. nlohmann::json importResultFile(std::string const& fileUrl, uint64_t rowCount, uint64_t fileSize); - /// &&&uj doc + /// Handle an error from the worker. nlohmann::json workerError(int errorCode, std::string const& errorMsg); std::ostream& dumpOS(std::ostream& os) const override; @@ -125,29 +128,29 @@ class UberJob : public JobBase { /// note: _jobsMtx must be locked before calling. bool _setStatusIfOk(qmeta::JobStatus::State newState, std::string const& msg); - /// &&&uj doc + /// unassign all Jobs in this UberJob and set the Executive flag to indicate that Jobs need + /// reassignment. void _unassignJobs(); - /// &&&uj doc - /// &&&uj The strings for errorType should have a centralized location in the code - global or util + /// 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); - /// &&&uj doc + /// Let the executive know that all Jobs in UberJob are complete. nlohmann::json _importResultFinish(uint64_t resultRows); - /// &&& uj doc + /// 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; //&&&uj + std::vector> _jobs; ///< List of Jobs in this UberJob. mutable std::mutex _jobsMtx; ///< Protects _jobs, _jobStatus std::atomic _started{false}; - bool _inSsi = false; - qmeta::JobStatus::Ptr _jobStatus{new qmeta::JobStatus()}; // &&&uj The JobStatus class should be changed - // to better represent UberJobs + 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. //&&&uj remove when possible + std::string _payload; ///< XrdSsi message to be sent to the _workerResource. TODO:UJ remove when possible std::weak_ptr _executive; std::shared_ptr _respHandler; @@ -156,12 +159,12 @@ class UberJob : public JobBase { qmeta::CzarId const _czarId; std::string const _idStr; - std::shared_ptr _qdispPool; //&&&uj needed? + std::shared_ptr _qdispPool; // TODO:UJ remove when possible. - // &&&uj - czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // &&& check if this is needed + // Map of workerData + czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // TODO:UJ this may not be needed - // &&&uj + // Contact information for the target worker. czar::CzarRegistry::WorkerContactInfo::Ptr _wContactInfo; }; diff --git a/src/qhttp/Server.cc b/src/qhttp/Server.cc index 466bf34c01..0cb23ae7ea 100644 --- a/src/qhttp/Server.cc +++ b/src/qhttp/Server.cc @@ -282,9 +282,6 @@ std::shared_ptr Server::_findPathHandler(Request::Ptr reque #else // &&& ===== LOGLS_DEBUG(_log, logger(this) << logger(request->_socket) << "invoking handler for " << pathHandler.path.regex); - LOGLS_DEBUG(_log, logger(this) << logger(request->_socket) << " &&&uj invoking handler for " - << pathHandler.path.regex << " " << request->path - << " match=" << pathMatch); try { pathHandler.handler(request, response); } catch (boost::system::system_error const& e) { diff --git a/src/qmeta/JobStatus.cc b/src/qmeta/JobStatus.cc index 3b5750ae3e..bf31305499 100644 --- a/src/qmeta/JobStatus.cc +++ b/src/qmeta/JobStatus.cc @@ -55,11 +55,6 @@ void JobStatus::updateInfo(std::string const& idMsg, JobStatus::State s, std::st std::string const& desc, MessageSeverity severity) { std::lock_guard lock(_mutex); _updateInfo(idMsg, s, source, code, desc, 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::_updateInfo(std::string const& idMsg, JobStatus::State s, std::string const& source, int code, diff --git a/src/qmeta/QMetaMysql.cc b/src/qmeta/QMetaMysql.cc index 8fff95cd5f..3535c66fea 100644 --- a/src/qmeta/QMetaMysql.cc +++ b/src/qmeta/QMetaMysql.cc @@ -883,9 +883,9 @@ QMetaChunkMap QMetaMysql::getChunkMap(chrono::time_point c 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_WARN, - "&&& QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table - << " chunk=" << chunk << " sz=" << size); + LOGS(_log, LOG_LVL_TRACE, + "QMetaInsrt{worker=" << worker << " dbN=" << database << " tblN=" << table + << " chunk=" << chunk << " sz=" << size); } chunkMap.updateTime = updateTime; } catch (exception const& ex) { @@ -917,11 +917,6 @@ chrono::time_point QMetaMysql::_getChunkMapUpdateTime(lock throw ConsistencyError(ERR_LOC, "Too many rows in result set of query " + query); } try { - int j = 0; // &&& del - for (auto const& str : updateTime) { // &&& del - LOGS(_log, LOG_LVL_WARN, "&&& _updatetime j=" << j << " Insrt=" << str << " stol=" << stol(str)); - ++j; - } 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/qproc/TaskMsgFactory.cc b/src/qproc/TaskMsgFactory.cc index 64f26b28ec..8a2d7434dc 100644 --- a/src/qproc/TaskMsgFactory.cc +++ b/src/qproc/TaskMsgFactory.cc @@ -56,7 +56,7 @@ using namespace std; namespace lsst::qserv::qproc { -//&&&uj - Probaly just delete this +// 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) { @@ -70,8 +70,7 @@ bool TaskMsgFactory::fillTaskMsg(proto::TaskMsg* taskMsg, ChunkQuerySpec const& taskMsg->set_jobid(jobId); taskMsg->set_attemptcount(attemptCount); taskMsg->set_czarid(czarId); - // LOGS(_log, LOG_LVL_INFO, "&&& _makeMsg ses=" << _session << " db=" << chunkQuerySpec.db << " qId=" << - // queryId << " jId=" << jobId << " att=" << attemptCount << " cz=" << czarId); + // scanTables (for shared scans) // check if more than 1 db in scanInfo std::string db; @@ -174,7 +173,6 @@ std::shared_ptr TaskMsgFactory::_makeMsg(ChunkQuerySpec const& c _addFragment(*taskMsg, resultTable, chunkQuerySpec.subChunkTables, chunkQuerySpec.subChunkIds, chunkQuerySpec.queries); } - LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::_makeMsg end"); return taskMsg; } @@ -216,26 +214,25 @@ std::shared_ptr TaskMsgFactory::makeMsgJson(ChunkQuerySpec const std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId) { - LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::makeMsgJson start"); std::string resultTable("Asdfasfd"); if (!chunkResultName.empty()) { - resultTable = - chunkResultName; // &&&uj this probably needs to be replaced with whatever it should be now. + resultTable = chunkResultName; } - 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()}})); // &&&uj verify that these can be put in the - // uberjob to reduce copies. + // 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; @@ -253,12 +250,11 @@ std::shared_ptr TaskMsgFactory::makeMsgJson(ChunkQuerySpec const ChunkQuerySpec const* sPtr = &chunkQuerySpec; while (sPtr) { LOGS(_log, LOG_LVL_TRACE, "nextFragment"); - for (unsigned int t = 0; t < (sPtr->queries).size(); t++) { // &&& delete block - LOGS(_log, LOG_LVL_TRACE, (sPtr->queries).at(t)); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC q=" << (sPtr->queries).at(t)); + 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) { // &&& delete block - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC sbi=" << sbi); + 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. @@ -275,7 +271,6 @@ std::shared_ptr TaskMsgFactory::makeMsgJson(ChunkQuerySpec const chunkQuerySpec.queries); } - LOGS(_log, LOG_LVL_WARN, "&&& TaskMsgFactory::makeMsg end " << jsJobMsg); return jsJobMsgPtr; } @@ -286,28 +281,12 @@ void TaskMsgFactory::_addFragmentJson(nlohmann::json& jsFragments, std::string c {"queries", nlohmann::json::array()}, {"subchunkTables", nlohmann::json::array()}, {"subchunkIds", nlohmann::json::array()}}; - LOGS(_log, LOG_LVL_WARN, "&&&SUBCa jsFrag=" << jsFrag); - - string tmp("&&&SUBCa queries="); - for (auto const& qstr : queries) { - tmp += "~"; - tmp += qstr + "~"; - } - LOGS(_log, LOG_LVL_WARN, __func__ << tmp); - - tmp = "&&&SUBCa scids="; - for (auto const& scid : subchunkIds) { - tmp += "~"; - tmp += to_string(scid) + "~"; - } - LOGS(_log, LOG_LVL_WARN, __func__ << tmp); auto& jsQueries = jsFrag["queries"]; for (auto& qry : queries) { nlohmann::json jsQry = {{"subQuery", qry}}; jsQueries.push_back(move(jsQry)); } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsQueries=" << jsQueries); // Add the db+table pairs to the subchunk. auto& jsSubchunkTables = jsFrag["subchunkTables"]; @@ -316,16 +295,13 @@ void TaskMsgFactory::_addFragmentJson(nlohmann::json& jsFragments, std::string c jsSubchunkTables.push_back(move(jsSubchunkTbl)); LOGS(_log, LOG_LVL_TRACE, "added dbtbl=" << tbl.db << "." << tbl.table); } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsSubchunkTables=" << jsSubchunkTables); // Add subchunk id numbers auto& jsSubchunkIds = jsFrag["subchunkIds"]; for (auto& subchunkId : subchunkIds) { jsSubchunkIds.push_back(subchunkId); } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsSubchunkIds=" << jsSubchunkIds); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBCz jsFrag=" << jsFrag); jsFragments.push_back(move(jsFrag)); } diff --git a/src/qproc/TaskMsgFactory.h b/src/qproc/TaskMsgFactory.h index d35ba9bdd5..d770d2c5c4 100644 --- a/src/qproc/TaskMsgFactory.h +++ b/src/qproc/TaskMsgFactory.h @@ -61,29 +61,28 @@ class TaskMsgFactory { virtual void serializeMsg(ChunkQuerySpec const& s, std::string const& chunkResultName, QueryId queryId, int jobId, int attemptCount, qmeta::CzarId czarId, std::ostream& os); - //&&&uj /// 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); - // &&& doc + /// 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: - // &&&uj probably delete + // 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); - // &&&uj probably delete + // 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); - /// &&& doc + /// 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); diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 357e2fafa7..11cb77cdd2 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -218,7 +218,7 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { void InfileMerger::mergeCompleteFor(int jobId) { std::lock_guard resultSzLock(_mtxResultSizeMtx); - _totalResultSize += _perJobResultSize[jobId]; //&&&uj this can probably be simplified + _totalResultSize += _perJobResultSize[jobId]; // TODO:UJ this can probably be simplified } bool InfileMerger::merge(proto::ResponseSummary const& responseSummary, @@ -334,13 +334,9 @@ bool InfileMerger::merge(proto::ResponseSummary const& responseSummary, } bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::ResponseData const& responseData) { - auto jq = uberJob; // &&& replace jq with uberJob - //&&&JobId const jobId = responseSummary.jobid(); UberJobId const uJobId = uberJob->getJobId(); - //&&&std::string queryIdJobStr = QueryIdHelper::makeIdStr(responseSummary.queryid(), jobId); std::string queryIdJobStr = uberJob->getIdStr(); if (!_queryIdStrSet) { - //&&&_setQueryIdStr(QueryIdHelper::makeIdStr(responseSummary.queryid())); _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); } @@ -350,10 +346,10 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response } // Do nothing if the query got cancelled for any reason. - if (jq->isQueryCancelled()) { + if (uberJob->isQueryCancelled()) { return true; } - auto executive = jq->getExecutive(); + auto executive = uberJob->getExecutive(); if (executive == nullptr || executive->getCancelled() || executive->isLimitRowComplete()) { return true; } @@ -378,7 +374,6 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, proto::Response util::Timer virtFileT; virtFileT.start(); // UberJobs only get one attempt - //&&&int resultJobId = makeJobIdAttempt(responseSummary.jobid(), responseSummary.attemptcount()); int resultJobId = makeJobIdAttempt(uberJob->getJobId(), 0); ProtoRowBuffer::Ptr pRowBuffer = std::make_shared( responseData, resultJobId, _jobIdColName, _jobIdSqlType, _jobIdMysqlType); diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index cf9e5d0581..d8e472c54b 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -168,8 +168,7 @@ class InfileMerger { bool merge(proto::ResponseSummary const& responseSummary, proto::ResponseData const& responseData, std::shared_ptr const& jq); - /// &&&uj doc - + /// 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. diff --git a/src/wbase/FileChannelShared.cc b/src/wbase/FileChannelShared.cc index f35aab9e77..722d4ea0c6 100644 --- a/src/wbase/FileChannelShared.cc +++ b/src/wbase/FileChannelShared.cc @@ -283,7 +283,6 @@ FileChannelShared::FileChannelShared(shared_ptr const& sendC } } -//&&&uj FileChannelShared::Ptr FileChannelShared::create(std::shared_ptr const& uberJob, qmeta::CzarId czarId, string const& czarHostName, int czarPort, string const& workerId) { @@ -305,13 +304,11 @@ FileChannelShared::FileChannelShared(std::shared_ptr const& _protobufArena(make_unique()), _scsId(scsSeqId++), _useHttp(true) { - LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created scsId=" << _scsId); - LOGS(_log, LOG_LVL_WARN, - "&&& FileChannelShared created useHttp true scsId=" << _scsId << " ujId=" << _uberJobId); + LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared created scsId=" << _scsId << " ujId=" << _uberJobId); } FileChannelShared::~FileChannelShared() { - LOGS(_log, LOG_LVL_WARN, "&&& ~FileChannelShared scsId=" << _scsId << " ujId=" << _uberJobId); + 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 @@ -326,10 +323,8 @@ FileChannelShared::~FileChannelShared() { _sendChannel->kill("~FileChannelShared()"); } } - } else { - LOGS(_log, LOG_LVL_WARN, "&&&uj should anything be sent to czar at this point???"); } - LOGS(_log, LOG_LVL_DEBUG, "FileChannelShared deleted"); + LOGS(_log, LOG_LVL_DEBUG, "~FileChannelShared end"); } void FileChannelShared::setTaskCount(int taskCount) { _taskCount = taskCount; } @@ -362,8 +357,6 @@ string FileChannelShared::makeIdStr(int qId, int jId) { bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared_ptr const& task, bool cancelled) { - LOGS(_log, LOG_LVL_WARN, - "&&& FileChannelShared::buildAndTransmitError scsId=" << _scsId << " ujId=" << _uberJobId); lock_guard const tMtxLock(_tMtx); if (!_useHttp) { if (!_sendResponse(tMtxLock, task, cancelled, multiErr)) { @@ -372,7 +365,6 @@ bool FileChannelShared::buildAndTransmitError(util::MultiError& multiErr, shared } return true; } else { - LOGS(_log, LOG_LVL_WARN, "&&&uj send msg to czar with the errors"); auto ujData = _uberJobData.lock(); if (ujData == nullptr) { LOGS(_log, LOG_LVL_WARN, @@ -450,8 +442,6 @@ bool FileChannelShared::buildAndTransmitResult(MYSQL_RES* mResult, shared_ptr const& streamMutexLock, string c if (!oldVal) { LOGS(_log, LOG_LVL_WARN, "FileChannelShared first kill call " << note); } - // &&&uj anything else need to be done? _removeFile(streamMutexLock); return oldVal; } @@ -526,10 +515,7 @@ bool FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p LOGS(_log, LOG_LVL_TRACE, __func__ << " file write " << task->getIdStr() << " start"); // Create the file if not open. if (!_file.is_open()) { - auto oldname = _fileName; // &&& del _fileName = task->resultFilePath(); - LOGS(_log, LOG_LVL_WARN, - "&&& opening file oldName=" << oldname << " new=" << _fileName << " scsId=" << _scsId); _file.open(_fileName, ios::out | ios::trunc | ios::binary); if (!(_file.is_open() && _file.good())) { throw runtime_error("FileChannelShared::" + string(__func__) + @@ -543,7 +529,6 @@ bool FileChannelShared::_writeToFile(lock_guard const& tMtxLock, shared_p uint32_t const msgSizeBytes = msg.size(); _file.write(reinterpret_cast(&msgSizeBytes), sizeof msgSizeBytes); _file.write(msg.data(), msgSizeBytes); - LOGS(_log, LOG_LVL_WARN, "&&&uj headerCount=" << _headerCount << " wrote msgSizeBytes=" << msgSizeBytes); if (!(_file.is_open() && _file.good())) { throw runtime_error("FileChannelShared::" + string(__func__) + " failed to write " + @@ -583,7 +568,7 @@ bool FileChannelShared::_fillRows(lock_guard const& tMtxLock, MYSQL_RES* } void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { - LOGS(_log, LOG_LVL_WARN, "&&& FileChannelShared::_removeFile " << _fileName << " scsId=" << _scsId); + LOGS(_log, LOG_LVL_TRACE, "FileChannelShared::_removeFile " << _fileName << " scsId=" << _scsId); if (!_fileName.empty()) { if (_file.is_open()) { _file.close(); @@ -603,8 +588,6 @@ void FileChannelShared::_removeFile(lock_guard const& tMtxLock) { bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ptr const& task, bool cancelled, util::MultiError const& multiErr) { - LOGS(_log, LOG_LVL_WARN, - "&&& FileChannelShared::_sendResponse scsId=" << _scsId << " task=" << task->getIdStr()); auto const queryId = task->getQueryId(); auto const jobId = task->getJobId(); auto const idStr(makeIdStr(queryId, jobId)); @@ -661,10 +644,6 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ 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) { // 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); @@ -681,59 +660,15 @@ bool FileChannelShared::_sendResponse(lock_guard const& tMtxLock, shared_ return false; } } else { - LOGS(_log, LOG_LVL_WARN, - "&&&uj send the url back with http scsId=" << _scsId << " task=" << task->getIdStr()); - // &&&uj the http communications need to happen in a different thread, or this thread can be booted - // &&&uj from the scheduler so that it can just wait for a response. 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(); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&uj ujId=" << _uberJobId << " scsId=" << _scsId << " task=" << task->getIdStr() - << " httpFileUrl=" << httpFileUrl); ujData->responseFileReady(httpFileUrl, _rowcount, _transmitsize, _headerCount); } return true; } -/* &&& -void FileChannelShared::_fileReadyResponse() { - json request = {{"version", http::MetaModule::version}, - {"workerid", _comInfoToCzar->foreman->chunkInventory()->id()}, - {"auth_key", authKey()}, - {"czar", czarName}, - {"czarid", czarId}, - {"queryid", queryId}, - {"uberjobid", uberJobId}}; - - auto const method = http::Method::POST; - vector const headers = {"Content-Type: application/json"}; - string const url = "http://" + czarHostName + ":" + to_string(czarPort) + "/queryjob-ready"; - string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - http::Client client(method, url, request.dump(), headers); - bool transmitSuccess = false; - try { - json const response = client.readAsJson(); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj response=" << response); - if (0 != response.at("success").get()) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj success"); - transmitSuccess = true; - } else { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE success=0"); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); - } - if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, - __func__ << "&&&uj NEED CODE try again??? Let czar find out through polling worker status???"); - } else { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); - } -} -*/ - } // namespace lsst::qserv::wbase diff --git a/src/wbase/FileChannelShared.h b/src/wbase/FileChannelShared.h index 8d6069fc8c..102f87fe24 100644 --- a/src/wbase/FileChannelShared.h +++ b/src/wbase/FileChannelShared.h @@ -56,12 +56,6 @@ namespace lsst::qserv::util { class MultiError; } // namespace lsst::qserv::util -/* &&& -namespace lsst::qserv::wcontrol { -class Foreman; -} -*/ - namespace lsst::qserv::wbase { class UberJobData; @@ -122,21 +116,14 @@ class FileChannelShared { */ static nlohmann::json filesToJson(std::vector const& queryIds, unsigned int maxFiles); - //&&&uj /// The factory method for the channel class. 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(UberJobId uberJobId, qmeta::CzarId czarId, std::string const& czarHostName, - int czarPort, std::string const& workerId, - ComInfoToCzar::Ptr const& comInfoToCzar); - */ /// 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); //&&& delete all params except uberJob + std::string const& workerId); // TODO:UJ delete all params except uberJob FileChannelShared() = delete; FileChannelShared(FileChannelShared const&) = delete; @@ -185,11 +172,11 @@ class FileChannelShared { bool isDead(); private: - //&&&uj - /// 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); @@ -251,21 +238,18 @@ class FileChannelShared { bool _sendResponse(std::lock_guard const& tMtxLock, std::shared_ptr const& task, bool cancelled, util::MultiError const& multiErr); - /// &&&uj doc - void _fileReadyResponse(); - 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; ///< &&& doc + std::weak_ptr _uberJobData; ///< Pointer to UberJobData - UberJobId const _uberJobId; ///< &&& doc - qmeta::CzarId const _czarId; ///< id of the czar that requested this task(s). &&& delete - std::string const _czarHostName; ///< Name of the czar host. &&& delete - int const _czarPort; ///< port for the czar. &&& delete - std::string const _workerId; ///< The unique identifier of the worker. &&& delete + 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 // Allocatons/deletion of the data messages are managed by Google Protobuf Arena. std::unique_ptr _protobufArena; @@ -305,9 +289,7 @@ 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; ///< &&& del - - //&&&ComInfoToCzar::Ptr _comInfoToCzar; ///< &&&uj doc + 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. diff --git a/src/wbase/Task.cc b/src/wbase/Task.cc index a9a46ad2f7..4e4153b812 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -81,8 +81,6 @@ string buildResultFilePath(shared_ptr const& taskMs return path.string(); } -// std::atomic fileSeq = 0; &&& -//&&&uj string buildUjResultFilePath(lsst::qserv::wbase::UberJobData::Ptr const& ujData, string const& resultsDirname) { if (resultsDirname.empty()) return resultsDirname; @@ -117,7 +115,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); @@ -148,7 +145,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 @@ -174,13 +170,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) { @@ -255,8 +244,6 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun _attemptCount(attemptCount), _queryFragmentNum(fragmentNumber), _fragmentHasSubchunks(hasSubchunks), - //&&&_hasDb(t->has_db()), - //&&&_db(t->has_db() ? t->db() : ""), _db(db), _czarId(ujData->getCzarId()), _scanInfo(scanInfo), @@ -269,13 +256,13 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun _resultFilePath = ::buildUjResultFilePath(ujData, workerConfig->resultsDirname()); auto const fqdn = util::get_current_host_fqdn(); if (resultDeliveryProtocol == wconfig::ConfigValResultDeliveryProtocol::HTTP) { - // &&&uj it seems like this should just be part of the FileChannelShared??? + // 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; //&&&uj + user = defaultUser; // Create sets and vectors for 'aquiring' subchunk temporary tables. // Fill in _dbTblsAndSubchunks @@ -284,8 +271,6 @@ 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. - LOGS(_log, LOG_LVL_WARN, "&&& Task::Task not _fragmentHasSubchunks"); - //&&&for (auto const& scanTbl : t->scantable()) { for (auto const& scanTbl : scanInfo.infoTables) { dbTbls_.emplace(scanTbl.db, scanTbl.table); LOGS(_log, LOG_LVL_INFO, @@ -294,50 +279,25 @@ Task::Task(UberJobData::Ptr const& ujData, int jobId, int attemptCount, int chun 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++) { - LOGS(_log, LOG_LVL_WARN, "&&& Task::Task true _fragmentHasSubchunks"); 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(sc.dbtbl(j).db(), sc.dbtbl(j).tbl()); dbTbls_.emplace(fDbTbl.db, fDbTbl.tbl); LOGS(_log, LOG_LVL_TRACE, "Task::Task subchunk fDbTbl.db=" << fDbTbl.db << " fDbTbl.tbl=" << fDbTbl.tbl); } - //&&& IntVector sVect(sc.id().begin(), sc.id().end()); - //&&&subchunksVect_ = sVect; - // &&&uj Is the entire subchunk vector needed here, or would one subchunk this query uses be enough. - // &&&uj Need to test subchunksVect_.push_back(subchunkId);. subchunksVect_ = fragSubchunkIds; - /* &&&uj _db set above, sc.database() is unused worker.proto - 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_)); - LOGS(_log, LOG_LVL_WARN, - "&&& fragment b db=" << _db << ":" << _chunkId << " dbTableSet" << util::printable(dbTbls_) - << " subChunks=" << util::printable(subchunksVect_)); } _dbTblsAndSubchunks = make_unique(dbTbls_, subchunksVect_); - LOGS(_log, LOG_LVL_WARN, "&&&uj Task::Task UberJob version end."); } Task::~Task() { - /* &&& - allIds.remove(to_string(_qId) + "_" + to_string(_jId)); - LOGS(_log, LOG_LVL_TRACE, "~Task() : " << allIds); - */ - _userQueryInfo.reset(); UserQueryInfo::uqMapErase(_qId); if (UserQueryInfo::uqMapGet(_qId) == nullptr) { @@ -390,7 +350,6 @@ vector Task::createTasks(shared_ptr const& taskMsg, } sendChannel->setTaskCount(vect.size()); - LOGS(_log, LOG_LVL_WARN, "&&& Task::createTasksForChunk end vect.sz=" << vect.size()); return vect; } @@ -403,7 +362,6 @@ std::vector Task::createTasksForChunk( std::shared_ptr const& queriesAndChunks, uint16_t resultsHttpPort) { QueryId qId = ujData->getQueryId(); UberJobId ujId = ujData->getUberJobId(); - LOGS(_log, LOG_LVL_WARN, "&&&uj Task::createTasksForChunk start"); UserQueryInfo::Ptr userQueryInfo = UserQueryInfo::uqMapInsert(qId); @@ -414,108 +372,76 @@ std::vector Task::createTasksForChunk( for (auto const& job : jsJobs) { json const& jsJobDesc = job["jobdesc"]; http::RequestBody rbJobDesc(jsJobDesc); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC jobdesc " << jsJobDesc); // See qproc::TaskMsgFactory::makeMsgJson for message construction. - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k1"); auto const jdCzarId = rbJobDesc.required("czarId"); - // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k2"); 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)); } - // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k3"); auto const jdJobId = rbJobDesc.required("jobId"); - // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k4"); auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k5"); auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - // LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k6"); auto const jdScanPriority = rbJobDesc.required("scanPriority"); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); auto const jdScanInteractive = rbJobDesc.required("scanInteractive"); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); auto const jdMaxTableSizeMb = rbJobDesc.required("maxTableSize"); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); auto const jdChunkId = rbJobDesc.required("chunkId"); - LOGS(_log, LOG_LVL_WARN, - funcN << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + 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); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); auto const jdQueryFragments = rbJobDesc.required("queryFragments"); - int fragmentNumber = 0; //&&&uj should this be 1??? Is this at all useful? + int fragmentNumber = 0; for (auto const& frag : jdQueryFragments) { vector fragSubQueries; vector fragSubchunkIds; vector fragSubTables; - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10a"); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC frag=" << frag); + LOGS(_log, LOG_LVL_DEBUG, funcN << " frag=" << frag); http::RequestBody rbFrag(frag); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10b"); auto const& jsQueries = rbFrag.required("queries"); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c"); - // &&&uj move to uberjob???, these should be the same for all jobs + // TODO:UJ move to uberjob???, these should be the same for all jobs for (auto const& subQ : jsQueries) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c1"); http::RequestBody rbSubQ(subQ); auto const subQuery = rbSubQ.required("subQuery"); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10c2"); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC subQuery=" << subQuery); + LOGS(_log, LOG_LVL_DEBUG, funcN << " subQuery=" << subQuery); fragSubQueries.push_back(subQuery); } - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k10d1"); auto const& resultTable = rbFrag.required("resultTable"); - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10d2"); auto const& jsSubIds = rbFrag.required("subchunkIds"); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scId jsSubIds=" << jsSubIds); for (auto const& scId : jsSubIds) { - // LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10e1"); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scId=" << scId); fragSubchunkIds.push_back(scId); } auto const& jsSubTables = rbFrag.required("subchunkTables"); - for (auto const& scDbTable : jsSubTables) { // &&&uj are these the same for all jobs? + for (auto const& scDbTable : jsSubTables) { // TODO:UJ are these the same for all jobs? http::RequestBody rbScDbTable(scDbTable); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC k10f1"); string scDb = rbScDbTable.required("scDb"); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scDb=" << scDb); string scTable = rbScDbTable.required("scTable"); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC scTable=" << scDbTable); TaskDbTbl scDbTbl(scDb, scTable); fragSubTables.push_back(scDbTbl); } - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC fragSubQueries.sz=" << fragSubQueries.size()); 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)); // &&& change to make_shared + 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); - LOGS(_log, LOG_LVL_WARN, - funcN << "&&&SUBC fragSubchunkIds.empty()==true vect.sz=" << vect.size() - << " fragNum=" << fragmentNumber); } 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)); // &&& change to make_shared + 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); - LOGS(_log, LOG_LVL_WARN, - funcN << "&&&SUBC fragSubchunkIds.empty()==false vect.sz=" << vect.size() - << " fragNum=" << fragmentNumber); } } } @@ -523,16 +449,11 @@ std::vector Task::createTasksForChunk( } } - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC vect.sz=" << vect.size()); - for (auto taskPtr : vect) { - LOGS(_log, LOG_LVL_WARN, funcN << "&&&SUBC taskPtr calling setTaskQueryRunner"); // newQueryRunner sets the `_taskQueryRunner` pointer in `task`. taskPtr->setTaskQueryRunner(wdb::QueryRunner::newQueryRunner(taskPtr, chunkResourceMgr, mySqlConfig, sqlConnMgr, queriesAndChunks)); } - // sendChannel->setTaskCount(vect.size()); &&& done at uberjob level now - LOGS(_log, LOG_LVL_WARN, "&&&uj Task::createTasksForChunk end vect.sz=" << vect.size()); return vect; } diff --git a/src/wbase/Task.h b/src/wbase/Task.h index 50a43de0c9..b6f57456e4 100644 --- a/src/wbase/Task.h +++ b/src/wbase/Task.h @@ -88,7 +88,7 @@ class TaskQueryRunner { virtual void cancel() = 0; ///< Repeated calls to cancel() must be harmless. }; -//&&&uj +/// Class for storing database + table name. class TaskDbTbl { public: TaskDbTbl() = delete; @@ -119,7 +119,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 { // &&&uj no longer needed +struct IdSet { // TODO:UJ delete if possible void add(std::string const& id) { std::lock_guard lock(mx); _ids.insert(id); @@ -174,12 +174,12 @@ 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); - // &&&uj too many parameters. - // &&&uj fragmentNumber seems pointless - // &&&uj hasSubchunks seems redundant. - // &&&uj Hopefully, many are the same for all tasks and can be moved to ujData and userQueryInfo. - // &&&uj Candidates: scanInfo, maxTableSizeMb, FileChannelShared, resultsHttpPort. - // &&&uj Unfortunately, this will be much easier if it is done after xrootd method is removed. + // 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, @@ -244,7 +244,6 @@ class Task : public util::CommandForThreadPool { TaskState state() const { return _state; } std::string getQueryString() const; - //&&&int getQueryFragmentNum() { return _queryFragmentNum; } std::string const& resultFilePath() const { return _resultFilePath; } std::string const& resultFileXrootUrl() const { return _resultFileXrootUrl; } std::string const& resultFileHttpUrl() const { return _resultFileHttpUrl; } @@ -364,9 +363,8 @@ 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. + std::string _db; ///< Task database + int const _czarId; ///< czar Id from the task message. /// Set of tables and vector of subchunk ids used by ChunkResourceRequest. Do not change/reset. std::unique_ptr _dbTblsAndSubchunks; diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 9fa6a0b036..598727a372 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -77,9 +77,9 @@ 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_WARN, - funcN << "&&& UberJobData::responseFileReady a httpFileUrl=" << httpFileUrl << " rows=" << rowCount - << " fSize=" << fileSize << " headerCount=" << headerCount); + LOGS(_log, LOG_LVL_TRACE, + funcN << " httpFileUrl=" << httpFileUrl << " rows=" << rowCount << " fSize=" << fileSize + << " headerCount=" << headerCount); json request = {{"version", http::MetaModule::version}, {"workerid", _foreman->chunkInventory()->id()}, @@ -93,53 +93,40 @@ void UberJobData::responseFileReady(string const& httpFileUrl, uint64_t rowCount {"fileSize", fileSize}, {"headerCount", headerCount}}; - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseFileReady b"); - auto const method = http::Method::POST; vector const headers = {"Content-Type: application/json"}; - //&&&string const url = "http://" + _czarName + ":" + to_string(_czarPort) + "/queryjob-ready"; string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-ready"; string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - LOGS(_log, LOG_LVL_WARN, - funcN + "&&&uj UberJobData::responseFileReady url=" << url << " request=" << request.dump()); http::Client client(method, url, request.dump(), headers); - int maxTries = 2; // &&& set from config + int maxTries = 2; // TODO:UJ set from config bool transmitSuccess = false; for (int j = 0; (!transmitSuccess && j < maxTries); ++j) { - LOGS(_log, LOG_LVL_WARN, - funcN + "&&&uj UberJobData::responseFileReady j=" << j << " url=" << url - << " request=" << request.dump()); try { json const response = client.readAsJson(); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj response=" << response); if (0 != response.at("success").get()) { - LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj success"); transmitSuccess = true; } else { - LOGS(_log, LOG_LVL_WARN, - funcN << "&&&uj NEED CODE success=0, result file should probably be deleted."); + 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 + " &&&uj failed, ex: " + ex.what()); + LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " failed, ex: " + ex.what()); } } if (!transmitSuccess) { LOGS(_log, LOG_LVL_ERROR, - funcN << "&&&uj NEED CODE Let czar find out through polling worker status??? Just throw the " + funcN << "TODO:UJ NEED CODE Let czar find out through polling worker status??? Just throw the " "result away???"); } - LOGS(_log, LOG_LVL_WARN, funcN << "&&& UberJobData::responseFileReady end"); } bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const& task, bool cancelled) { - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseError a"); string const funcN = cName(__func__); - + LOGS(_log, LOG_LVL_INFO, funcN); string errorMsg; int errorCode = 0; if (!multiErr.empty()) { @@ -165,34 +152,26 @@ bool UberJobData::responseError(util::MultiError& multiErr, std::shared_ptr const headers = {"Content-Type: application/json"}; - //&&&string const url = "http://" + _czarName + ":" + to_string(_czarPort) + "/queryjob-ready"; string const url = "http://" + _czarHost + ":" + to_string(_czarPort) + "/queryjob-error"; string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - LOGS(_log, LOG_LVL_WARN, "&&& UberJobData::responseError c"); - LOGS(_log, LOG_LVL_WARN, - funcN << "&&&uj UberJobData::responseError url=" << url << " request=" << request.dump()); http::Client client(method, url, request.dump(), headers); - int maxTries = 2; // &&& set from config + 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(); - LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj response=" << response); if (0 != response.at("success").get()) { - LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj success"); transmitSuccess = true; } else { - LOGS(_log, LOG_LVL_WARN, funcN << "&&&uj NEED CODE success=0"); + 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 + " &&&uj failed, ex: " + ex.what()); + LOGS(_log, LOG_LVL_WARN, funcN + " " + requestContext + " failed, ex: " + ex.what()); } } return transmitSuccess; diff --git a/src/wbase/UberJobData.h b/src/wbase/UberJobData.h index 8316eadee6..f4ab4e3030 100644 --- a/src/wbase/UberJobData.h +++ b/src/wbase/UberJobData.h @@ -53,8 +53,8 @@ namespace lsst::qserv::wbase { class FileChannelShared; class Task; -// &&&uj doc -/// This class tracks all Tasks associates with the UberJob and reports status to the czar. +/// 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; @@ -69,7 +69,7 @@ class UberJobData { return Ptr(new UberJobData(uberJobId, czarName, czarId, czarHost, czarPort, queryId, workerId, foreman, authKey)); } - // &&& doc + /// Set file channel for this UberJob void setFileChannelShared(std::shared_ptr const& fileChannelShared); UberJobId getUberJobId() const { return _uberJobId; } @@ -79,16 +79,16 @@ class UberJobData { uint64_t getQueryId() const { return _queryId; } std::string getWorkerId() const { return _workerId; } - /// &&& doc + /// 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()); } - /// &&& doc + /// Let the czar know the result is ready. void responseFileReady(std::string const& httpFileUrl, uint64_t rowCount, uint64_t fileSize, - uint64_t headerCount); // &&& remove headerCount + uint64_t headerCount); // TODO:UJ remove headerCount - /// &&& doc + /// 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; } @@ -105,7 +105,7 @@ class UberJobData { std::string const _czarHost; int const _czarPort; QueryId const _queryId; - std::string const _workerId; //&&&uj should be able to get this from the worker in a reasonable way. + std::string const _workerId; std::string const _authKey; std::shared_ptr const _foreman; @@ -113,15 +113,6 @@ class UberJobData { std::vector> _ujTasks; std::shared_ptr _fileChannelShared; - //&&&std::shared_ptr const foreman; - //&&& std::string const targetWorkerId; _workerId - //&&&std::string const czarName; - //&&&qmeta::CzarId const czarId; - //&&&std::string const czarHostName; _czarHost - //&&& int const czarPort; - //&&& uint64_t const queryId; - //&&&uint64_t const uberJobId; - std::string const _idStr; }; diff --git a/src/wbase/UserQueryInfo.h b/src/wbase/UserQueryInfo.h index 5299985a94..4b7a799f03 100644 --- a/src/wbase/UserQueryInfo.h +++ b/src/wbase/UserQueryInfo.h @@ -65,7 +65,7 @@ class UserQueryInfo { /// @throws Bug if id is out of range. std::string getTemplate(size_t id); - /// &&& doc + /// Add an UberJobData object to the UserQueryInfo. void addUberJob(std::shared_ptr const& ujData); private: diff --git a/src/wdb/QueryRunner.cc b/src/wdb/QueryRunner.cc index be3224d8eb..a4a7557ab5 100644 --- a/src/wdb/QueryRunner.cc +++ b/src/wdb/QueryRunner.cc @@ -218,7 +218,6 @@ bool QueryRunner::runQuery() { MYSQL_RES* QueryRunner::_primeResult(string const& query) { util::HoldTrack::Mark mark(ERR_LOC, "QR _primeResult() QID=" + _task->getIdStr()); - LOGS(_log, LOG_LVL_WARN, "&&& " << _task->getIdStr() << " QueryRunner::_primeResult " << query); bool queryOk = _mysqlConn->queryUnbuffered(query); if (!queryOk) { sql::SqlErrorObject errObj; diff --git a/src/wsched/ChunkTasksQueue.cc b/src/wsched/ChunkTasksQueue.cc index 088ad8d18e..de2a09bbbc 100644 --- a/src/wsched/ChunkTasksQueue.cc +++ b/src/wsched/ChunkTasksQueue.cc @@ -37,37 +37,6 @@ LOG_LOGGER _log = LOG_GET("lsst.qserv.wsched.ChunkTasksQueue"); namespace lsst::qserv::wsched { -/* &&& -/// Queue a Task with other tasks on the same chunk. -void ChunkTasksQueue::queueTask(std::vector const& tasks) { - std::lock_guard lg(_mapMx); - auto iter = _chunkMap.end(); - 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) { - iter = _chunkMap.find(chunkId); - if (iter == _chunkMap.end()) { - // Correct ChunkTask wasn't found, make a new one. - std::pair ele(chunkId, std::make_shared(chunkId, _memMan)); - auto res = _chunkMap.insert(ele); // insert should fail if the key already exists. - LOGS(_log, LOG_LVL_DEBUG, " queueTask chunk=" << chunkId << " created=" << res.second); - iter = res.first; - } - } - ++_taskCount; - iter->second->queTask(task); - } -} -*/ - /// Queue tasks from an uberjob. void ChunkTasksQueue::queueTask(std::vector const& tasks) { std::lock_guard lg(_mapMx); @@ -75,14 +44,6 @@ void ChunkTasksQueue::queueTask(std::vector const& tasks) { 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. UberJobs are constructed in a way that makes it likely // that subchunks for the same chunk will be grouped together in `tasks`. diff --git a/src/wsched/ScanScheduler.cc b/src/wsched/ScanScheduler.cc index e4410a1d62..06a489c855 100644 --- a/src/wsched/ScanScheduler.cc +++ b/src/wsched/ScanScheduler.cc @@ -234,7 +234,6 @@ void ScanScheduler::queCmd(vector const& cmds) { jid = tsk->getJobId(); QSERV_LOGCONTEXT_QUERY_JOB(qid, jid); } else { - // &&&if (qid != t->getQueryId() || jid != 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()); diff --git a/src/xrdsvc/HttpReplicaMgtModule.cc b/src/xrdsvc/HttpReplicaMgtModule.cc index ca3a06244a..14fdde32af 100644 --- a/src/xrdsvc/HttpReplicaMgtModule.cc +++ b/src/xrdsvc/HttpReplicaMgtModule.cc @@ -36,7 +36,6 @@ #include "http/RequestBodyJSON.h" #include "http/RequestQuery.h" #include "mysql/MySqlUtils.h" -//&&& #include "qmeta/types.h" #include "util/String.h" #include "wconfig/WorkerConfig.h" #include "wcontrol/Foreman.h" diff --git a/src/xrdsvc/HttpSvc.cc b/src/xrdsvc/HttpSvc.cc index 41fd85335b..49781fc24e 100644 --- a/src/xrdsvc/HttpSvc.cc +++ b/src/xrdsvc/HttpSvc.cc @@ -135,7 +135,7 @@ uint16_t HttpSvc::start() { HttpReplicaMgtModule::process(::serviceName, self->_foreman, req, resp, "REBUILD", http::AuthType::REQUIRED); }}}); - _httpServerPtr->addHandlers( //&&&uj + _httpServerPtr->addHandlers( {{"POST", "/queryjob", [self](shared_ptr const& req, shared_ptr const& resp) { HttpWorkerCzarModule::process(::serviceName, self->_foreman, req, resp, "QUERYJOB", diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 4514dfb7fe..8c0748b5ae 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -31,13 +31,13 @@ #include "lsst/log/Log.h" // Qserv headers -#include "http/Client.h" // &&&uj will probably need to be removed +#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" // &&&uj +#include "qmeta/types.h" #include "util/String.h" #include "util/Timer.h" #include "wbase/FileChannelShared.h" @@ -51,8 +51,6 @@ #include "xrdsvc/SsiProvider.h" #include "xrdsvc/XrdName.h" -//&&&extern XrdSsiProvider* XrdSsiProviderLookup; - using namespace std; using json = nlohmann::json; @@ -64,7 +62,7 @@ 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. -// &&& These errors seem useful enought to be centralized ??? +// 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}}); @@ -88,16 +86,14 @@ HttpWorkerCzarModule::HttpWorkerCzarModule(string const& context, json HttpWorkerCzarModule::executeImpl(string const& subModuleName) { string const func = string(__func__) + "[sub-module='" + subModuleName + "']"; - debug(func + "&&&uj xrdsvc"); enforceInstanceId(func, wconfig::WorkerConfig::instance()->replicationInstanceId()); enforceWorkerId(func); if (subModuleName == "QUERYJOB") return _queryJob(); throw invalid_argument(context() + func + " unsupported sub-module"); } -json HttpWorkerCzarModule::_queryJob() { // &&&uj - //&&&debug(__func__); - debug(string(__func__) + " &&&uj _queryJob()"); +json HttpWorkerCzarModule::_queryJob() { + debug(__func__); checkApiVersion(__func__, 34); // At this point, API version, correct worker, and auth have been checked. json jsRet = _handleQueryJob(__func__); @@ -109,46 +105,28 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { vector ujTasks; try { // See qdisp::UberJob::runUberJob() for json message construction. - LOGS(_log, LOG_LVL_ERROR, __func__ << "&&&SUBC NEEDS CODE"); auto const& jsReq = body().objJson; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jsReq=" << jsReq); string const targetWorkerId = body().required("worker"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC targetWorkerId=" << targetWorkerId); - // &&& ??? Maybe add RequestBody(json const& js) constructor to leverage functions for nested items - // like "czar". - //&&&auto const& jsCzar = jsReq["czar"]; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC a"); http::RequestBody rbCzar(body().required("czar")); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC b"); auto czarName = rbCzar.required("name"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC c"); auto czarId = rbCzar.required("id"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC d"); auto czarPort = rbCzar.required("management-port"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC e"); auto czarHostName = rbCzar.required("management-host-name"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC f"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC czar n=" << czarName << " id=" << czarId << " p=" << czarPort + 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"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC g"); auto ujId = rbUberJob.required("uberjobid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC h"); auto ujCzarId = rbUberJob.required("czarid"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC i"); auto ujJobs = rbUberJob.required("jobs"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC j"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); + LOGS(_log, LOG_LVL_TRACE, + __func__ << " uj qid=" << ujQueryId << " ujid=" << ujId << " czid=" << ujCzarId); - //&&&uj make UberJobData, UseQueryInfo entry, FileChannelShared, and Tasks. auto ujData = wbase::UberJobData::create(ujId, czarName, czarId, czarHostName, czarPort, ujQueryId, targetWorkerId, foreman(), authKey()); - LOGS(_log, LOG_LVL_WARN, "&&&uj (ujData != nullptr) = " << (ujData != nullptr)); // Find the entry for this queryId, creat a new one if needed. wbase::UserQueryInfo::Ptr userQueryInfo = wbase::UserQueryInfo::uqMapInsert(ujQueryId); @@ -158,84 +136,58 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { 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; // &&& + proto::ScanInfo scanInfo; bool scanInfoSet = false; bool jdScanInteractive = false; int jdMaxTableSize = 0; - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k"); for (auto const& job : ujJobs) { json const& jsJobDesc = job["jobdesc"]; http::RequestBody rbJobDesc(jsJobDesc); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC jobdesc " << jsJobDesc); // See qproc::TaskMsgFactory::makeMsgJson for message construction. - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k1"); auto const jdCzarId = rbJobDesc.required("czarId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k2"); jdQueryId = rbJobDesc.required("queryId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k3"); auto const jdJobId = rbJobDesc.required("jobId"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k4"); auto const jdAttemptCount = rbJobDesc.required("attemptCount"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k5"); auto const jdQuerySpecDb = rbJobDesc.required("querySpecDb"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k6"); auto const jdScanPriority = rbJobDesc.required("scanPriority"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k7"); jdScanInteractive = rbJobDesc.required("scanInteractive"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k8"); jdMaxTableSize = rbJobDesc.required("maxTableSize"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k9"); auto const jdChunkId = rbJobDesc.required("chunkId"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC jd cid=" << jdCzarId << " jdQId=" << jdQueryId << " jdJobId=" << jdJobId + 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); - //&&&uj need scan table info befor making tasks - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11"); - //&&&proto::ScanTableInfo::ListOf scanTables; - //&&&proto::ScanInfo scanInfo; auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); if (!scanInfoSet) { for (auto const& tbl : jdChunkScanTables) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a1"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC tbl=" << tbl); http::RequestBody rbTbl(tbl); auto const& chunkScanDb = rbTbl.required("db"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a2"); auto const& lockInMemory = rbTbl.required("lockInMemory"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a3"); auto const& chunkScanTable = rbTbl.required("table"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a4"); auto const& tblScanRating = rbTbl.required("tblScanRating"); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k11a5"); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory + LOGS(_log, LOG_LVL_TRACE, + __func__ << " chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); scanInfo.infoTables.emplace_back(chunkScanDb, chunkScanTable, lockInMemory, tblScanRating); scanInfoSet = true; } } - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k12"); scanInfo.scanRating = jdScanPriority; - - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC k10"); } // create tasks and add them to ujData - auto chunkTasks = wbase::Task::createTasksForChunk( // &&& getting called twice when it should only - // be called once + auto chunkTasks = wbase::Task::createTasksForChunk( ujData, ujJobs, channelShared, scanInfo, jdScanInteractive, jdMaxTableSize, foreman()->chunkResourceMgr(), foreman()->mySqlConfig(), foreman()->sqlConnMgr(), foreman()->queriesAndChunks(), foreman()->httpPort()); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&&SUBC chunkTasks.sz=" << chunkTasks.size() << " QID=" << jdQueryId); ujTasks.insert(ujTasks.end(), chunkTasks.begin(), chunkTasks.end()); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC ujTasks.sz=" << ujTasks.size() << " QID=" << jdQueryId); channelShared->setTaskCount(ujTasks.size()); ujData->addTasks(ujTasks); @@ -244,16 +196,11 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { timer.start(); foreman()->processTasks(ujTasks); // Queues tasks to be run later. timer.stop(); - LOGS(_log, LOG_LVL_WARN, - __func__ << "&&& Enqueued UberJob time=" << timer.getElapsed() << " " << jsReq); - - // &&&uj temporary, send response back to czar saying file is ready. The file is not ready, but this - // is just an initial comms test - //&&&_temporaryRespFunc(targetWorkerId, czarName, czarId, czarHostName, czarPort, ujQueryId, ujId); + 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()); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&SUBC note=" << note); jsRet = {{"success", 1}, {"errortype", "none"}, {"note", note}}; } catch (wbase::TaskException const& texp) { @@ -263,46 +210,4 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { return jsRet; } -#if 0 //&&& -// &&&uj delete -void HttpWorkerCzarModule::_temporaryRespFunc(string const& targetWorkerId, string const& czarName, - qmeta::CzarId czarId, string const& czarHostName, int czarPort, - uint64_t queryId, uint64_t uberJobId) { - - json request = {{"version", http::MetaModule::version}, - {"workerid", foreman()->chunkInventory()->id()}, - {"auth_key", authKey()}, - {"czar", czarName}, - {"czarid", czarId}, - {"queryid", queryId}, - {"uberjobid", uberJobId}}; - - auto const method = http::Method::POST; - vector const headers = {"Content-Type: application/json"}; - string const url = "http://" + czarHostName + ":" + to_string(czarPort) + "/queryjob-error"; - string const requestContext = "Worker: '" + http::method2string(method) + "' request to '" + url + "'"; - LOGS(_log, LOG_LVL_WARN, "czarName=" << czarName << " czarHostName=" << czarHostName << " &&&uj HttpWorkerCzarModule::_temporaryRespFunc url=" << url << " request=" << request.dump()); - http::Client client(method, url, request.dump(), headers); - bool transmitSuccess = false; - try { - json const response = client.readAsJson(); - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj response=" << response); - if (0 != response.at("success").get()) { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj success"); - transmitSuccess = true; - } else { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE success=0"); - } - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " &&&uj failed, ex: " + ex.what()); - } - if (!transmitSuccess) { - LOGS(_log, LOG_LVL_ERROR, - __func__ << "&&&uj NEED CODE try again??? Let czar find out through polling worker status???"); - } else { - LOGS(_log, LOG_LVL_WARN, __func__ << "&&&uj NEED CODE do nothing, czar should collect file"); - } -} -#endif // &&& - } // namespace lsst::qserv::xrdsvc diff --git a/src/xrdsvc/HttpWorkerCzarModule.h b/src/xrdsvc/HttpWorkerCzarModule.h index 6be1dd4f1f..bb75a63c55 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.h +++ b/src/xrdsvc/HttpWorkerCzarModule.h @@ -50,7 +50,7 @@ class SsiProviderServer; // This header declarations namespace lsst::qserv::xrdsvc { -/// &&& doc +/// This class handles Http message from the czar to the worker. class HttpWorkerCzarModule : public xrdsvc::HttpModule { public: /// @note supported values for parameter 'subModuleName' are: @@ -75,19 +75,12 @@ class HttpWorkerCzarModule : public xrdsvc::HttpModule { std::shared_ptr const& req, std::shared_ptr const& resp); - /// &&& doc + /// Handle an UberJob message from the czar to run it on this worker by calling _handleQueryJob. nlohmann::json _queryJob(); - /// &&& doc + /// 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); - - /* &&& - /// &&&uj temporary function for testing communication. Something like this will - /// need to be called when the uberjob has finished making the result file. - void _temporaryRespFunc(std::string const& targetWorkerId, std::string const& czarName, - qmeta::CzarId czarId, std::string const& czarHostName, int czarPort, - uint64_t ujQueryId, uint64_t ujId); - */ }; } // namespace lsst::qserv::xrdsvc From d2163a708612ebb533e8903d54c9f640789eb08d Mon Sep 17 00:00:00 2001 From: John Gates Date: Mon, 22 Jul 2024 10:19:16 -0700 Subject: [PATCH 12/15] CzarFamilyMap create now waits for a usccessful read. --- src/czar/Czar.cc | 4 ++++ src/czar/CzarChunkMap.cc | 20 +++++++++++++++++++- src/czar/CzarChunkMap.h | 3 ++- 3 files changed, 25 insertions(+), 2 deletions(-) diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index f8434281c0..9014382fc8 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -241,11 +241,15 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->qMetaCzarId); + /* &&& try { _czarFamilyMap = CzarFamilyMap::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 database before the restart. diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 2570b3dc9a..166c6414be 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -246,11 +246,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()); diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index b6f69d38b2..82ce15c7bc 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -244,7 +244,8 @@ 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); //&&& { return Ptr(new CzarFamilyMap(qmeta)); } CzarFamilyMap() = delete; CzarFamilyMap(CzarFamilyMap const&) = delete; From fdcccbd583d11c539a222dc474075c5880dfc37b Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 23 Jul 2024 14:49:00 -0700 Subject: [PATCH 13/15] Changed version numer to 35. --- src/admin/python/lsst/qserv/admin/replicationInterface.py | 2 ++ src/czar/HttpCzarWorkerModule.cc | 1 + src/http/MetaModule.cc | 7 ++++++- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/admin/python/lsst/qserv/admin/replicationInterface.py b/src/admin/python/lsst/qserv/admin/replicationInterface.py index 88f8ebfbc4..322a2a0d84 100644 --- a/src/admin/python/lsst/qserv/admin/replicationInterface.py +++ b/src/admin/python/lsst/qserv/admin/replicationInterface.py @@ -201,6 +201,8 @@ def __init__( self.repl_ctrl = urlparse(repl_ctrl_uri) self.auth_key = auth_key self.admin_auth_key = admin_auth_key + + # Must match MetaModule::version in http/MetaModule.cc self.repl_api_version = 35 _log.debug(f"ReplicationInterface %s", self.repl_ctrl) diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 3d179d577f..74a3cea3fb 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -62,6 +62,7 @@ HttpCzarWorkerModule::HttpCzarWorkerModule(string const& context, shared_ptrreplicationInstanceId(); enforceCzarName(func); if (subModuleName == "QUERYJOB-ERROR") return _queryJobError(); diff --git a/src/http/MetaModule.cc b/src/http/MetaModule.cc index 622d268366..4422ade6dd 100644 --- a/src/http/MetaModule.cc +++ b/src/http/MetaModule.cc @@ -37,7 +37,12 @@ string const adminAuthKey; namespace lsst::qserv::http { -unsigned int const MetaModule::version = 35; // &&&uj +// MetaModule::version is the ultimate source of truth for the version number. +// All version values must match. Other version location are in : +// class ReplicationInterface repl_api_version +// in src/admin/python/lsst/qserv/admin/replicationinterface.py +// RestAPIVersion in src/www/qserv/js/Common.js +unsigned int const MetaModule::version = 35; // TODO:UJ this may need to change when merging the branch. void MetaModule::process(string const& context, nlohmann::json const& info, shared_ptr const& req, shared_ptr const& resp, From cd41d574a920c6167c225ce92b659f6a7fd7d842 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 30 Jul 2024 07:59:17 -0700 Subject: [PATCH 14/15] Changed comments. --- src/czar/Czar.cc | 22 ++++++++++------------ src/xrdsvc/HttpWorkerCzarModule.cc | 4 ++-- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 9014382fc8..64d41a3c25 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -177,10 +177,10 @@ void Czar::_monitor() { /// 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`. + // TODO:UJ DM-45470 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. @@ -205,7 +205,7 @@ void Czar::_monitor() { execVal->assignJobsToUberJobs(); } - // TODO:UJ Maybe get missing results from workers. + // TODO:UJ DM-45470 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 @@ -215,6 +215,11 @@ void Czar::_monitor() { // TODO:UJ Maybe send a list of cancelled and completed queries to the workers? // How long should queryId's remain on this list? + // It's probably better to have the executive for a query to send out + // messages to worker that a user query was cancelled. If a worker sends + // the czar about a cancelled user query, or the executive for that + // query cannot be found, the worker should cancel all Tasks associated + // with that queryId. } } @@ -241,13 +246,6 @@ Czar::Czar(string const& configFilePath, string const& czarName) // the name of the Czar gets translated into a numeric identifier. _czarConfig->setId(_uqFactory->userQuerySharedResources()->qMetaCzarId); - /* &&& - try { - _czarFamilyMap = CzarFamilyMap::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); diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 8c0748b5ae..2399ec5388 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -168,9 +168,9 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { for (auto const& tbl : jdChunkScanTables) { http::RequestBody rbTbl(tbl); auto const& chunkScanDb = rbTbl.required("db"); - auto const& lockInMemory = rbTbl.required("lockInMemory"); + auto lockInMemory = rbTbl.required("lockInMemory"); auto const& chunkScanTable = rbTbl.required("table"); - auto const& tblScanRating = rbTbl.required("tblScanRating"); + auto tblScanRating = rbTbl.required("tblScanRating"); LOGS(_log, LOG_LVL_TRACE, __func__ << " chunkSDb=" << chunkScanDb << " lockinmem=" << lockInMemory << " csTble=" << chunkScanTable << " tblScanRating=" << tblScanRating); From aff07e0c57031533fca46eb1ac9c047912501bf3 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 30 Jul 2024 15:43:31 -0700 Subject: [PATCH 15/15] Rebase. --- src/czar/CMakeLists.txt | 1 - src/czar/Czar.cc | 78 ------------------------------ src/czar/CzarChunkMap.h | 3 +- src/czar/HttpCzarWorkerModule.cc | 2 +- src/czar/HttpCzarWorkerModule.h | 4 +- src/http/Module.h | 3 +- src/http/RequestBodyJSON.h | 18 +++---- src/qhttp/Server.cc | 24 --------- src/wbase/Task.cc | 10 ++-- src/wbase/UberJobData.cc | 2 +- src/xrdsvc/HttpWorkerCzarModule.cc | 10 ++-- 11 files changed, 24 insertions(+), 131 deletions(-) diff --git a/src/czar/CMakeLists.txt b/src/czar/CMakeLists.txt index e10e4bea2b..9fb9f540a3 100644 --- a/src/czar/CMakeLists.txt +++ b/src/czar/CMakeLists.txt @@ -10,7 +10,6 @@ target_sources(czar PRIVATE HttpCzarQueryModule.cc HttpCzarSvc.cc HttpCzarWorkerModule.cc - HttpModule.cc HttpMonitorModule.cc HttpSvc.cc MessageTable.cc diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 64d41a3c25..bc73e2eca5 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -79,84 +79,6 @@ namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.czar.Czar"); -/** - * This function will keep periodically updating Czar's info in the Replication - * System's Registry. - * @param name The unique identifier of the Czar to be registered. - * @param czarConfig A pointer to the Czar configuration service. - * @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(shared_ptr const& czarConfig) { - 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({{"version", http::MetaModule::version}, - {"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_WARN, "&&&czarPost url=" << url); - LOGS(_log, LOG_LVL_WARN, "&&&czarPost request=" << request.dump()); - LOGS(_log, LOG_LVL_WARN, "&&&czarPost headers=" << headers[0]); - 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, czarConfig->replicationRegistryHearbeatIvalSec()))); - } -} - -// &&& doc -void registryWorkerInfoLoop(shared_ptr const& czarConfig) { - // Get worker information from the registry - auto const method = http::Method::GET; - string const url = - "http://" + czarConfig->replicationRegistryHost() + ":" + - to_string(czarConfig->replicationRegistryPort()) + "/services?instance_id=" + - czarConfig->replicationInstanceId(); // &&& what is this value supposed to be to get worker info? - vector const headers = {"Content-Type: application/json"}; - json request = nlohmann::json(); - string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; - LOGS(_log, LOG_LVL_WARN, "&&&czarGet url=" << url); - LOGS(_log, LOG_LVL_WARN, "&&&czarGet request=" << request.dump()); - LOGS(_log, LOG_LVL_WARN, "&&&czarGet headers=" << headers[0]); - http::Client client(method, url, request.dump(), headers); - while (true) { - LOGS(_log, LOG_LVL_WARN, "&&&czarGet loop start"); - 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(); - } - */ - LOGS(_log, LOG_LVL_WARN, "&&&czarGet resp=" << response); - } catch (exception const& ex) { - LOGS(_log, LOG_LVL_WARN, requestContext + " failed, ex: " + ex.what()); - LOGS(_log, LOG_LVL_WARN, requestContext + " &&& failed, ex: " + ex.what()); - } - this_thread::sleep_for(chrono::seconds(15)); - } -} - } // anonymous namespace namespace lsst::qserv::czar { diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index 82ce15c7bc..f0b85a1d31 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -244,8 +244,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; diff --git a/src/czar/HttpCzarWorkerModule.cc b/src/czar/HttpCzarWorkerModule.cc index 74a3cea3fb..471bacee2e 100644 --- a/src/czar/HttpCzarWorkerModule.cc +++ b/src/czar/HttpCzarWorkerModule.cc @@ -57,7 +57,7 @@ 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 + "']"; diff --git a/src/czar/HttpCzarWorkerModule.h b/src/czar/HttpCzarWorkerModule.h index ef0a745093..69f4a3fef4 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 diff --git a/src/http/Module.h b/src/http/Module.h index 2864496bf1..0a8d0ac957 100644 --- a/src/http/Module.h +++ b/src/http/Module.h @@ -208,8 +208,7 @@ class Module { */ virtual void sendResponse(std::string const& content, std::string const& contentType) = 0; - std::string authKey() const { return _authKey; } // &&&uj - + std::string authKey() const { return _authKey; } private: /** diff --git a/src/http/RequestBodyJSON.h b/src/http/RequestBodyJSON.h index fa738e8556..a3363e3416 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/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/wbase/Task.cc b/src/wbase/Task.cc index 4e4153b812..0448a6af77 100644 --- a/src/wbase/Task.cc +++ b/src/wbase/Task.cc @@ -46,7 +46,7 @@ #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 "util/Bug.h" @@ -371,7 +371,7 @@ std::vector Task::createTasksForChunk( 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"); @@ -399,11 +399,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); @@ -416,7 +416,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); diff --git a/src/wbase/UberJobData.cc b/src/wbase/UberJobData.cc index 598727a372..d969b80b7f 100644 --- a/src/wbase/UberJobData.cc +++ b/src/wbase/UberJobData.cc @@ -34,7 +34,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" diff --git a/src/xrdsvc/HttpWorkerCzarModule.cc b/src/xrdsvc/HttpWorkerCzarModule.cc index 2399ec5388..af6f741daf 100644 --- a/src/xrdsvc/HttpWorkerCzarModule.cc +++ b/src/xrdsvc/HttpWorkerCzarModule.cc @@ -34,7 +34,7 @@ #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 "qmeta/types.h" @@ -108,7 +108,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { auto const& jsReq = body().objJson; string const targetWorkerId = body().required("worker"); - http::RequestBody rbCzar(body().required("czar")); + http::RequestBodyJSON rbCzar(body().required("czar")); auto czarName = rbCzar.required("name"); auto czarId = rbCzar.required("id"); auto czarPort = rbCzar.required("management-port"); @@ -117,7 +117,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { __func__ << " czar n=" << czarName << " id=" << czarId << " p=" << czarPort << " h=" << czarHostName); - http::RequestBody rbUberJob(body().required("uberjob")); + http::RequestBodyJSON rbUberJob(body().required("uberjob")); auto ujQueryId = rbUberJob.required("queryid"); auto ujId = rbUberJob.required("uberjobid"); auto ujCzarId = rbUberJob.required("czarid"); @@ -146,7 +146,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { for (auto const& job : ujJobs) { 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"); jdQueryId = rbJobDesc.required("queryId"); @@ -166,7 +166,7 @@ json HttpWorkerCzarModule::_handleQueryJob(string const& func) { auto const jdChunkScanTables = rbJobDesc.required("chunkScanTables"); if (!scanInfoSet) { for (auto const& tbl : jdChunkScanTables) { - http::RequestBody rbTbl(tbl); + http::RequestBodyJSON rbTbl(tbl); auto const& chunkScanDb = rbTbl.required("db"); auto lockInMemory = rbTbl.required("lockInMemory"); auto const& chunkScanTable = rbTbl.required("table");