From 3e6ea561699ffecb00cebb0ee1acd3f5cfe8aa2a Mon Sep 17 00:00:00 2001 From: John Gates Date: Fri, 13 Jun 2025 11:38:14 -0700 Subject: [PATCH 01/11] Added CsvStream child that stores complete result in memory. --- src/cconfig/CzarConfig.h | 8 +- src/ccontrol/MergingHandler.cc | 27 ++++-- src/czar/Czar.cc | 14 ++++ src/mysql/CsvBuffer.cc | 97 ++++++++++++++++++++++ src/mysql/CsvBuffer.h | 147 +++++++++++++++++++++++++++++++-- src/rproc/InfileMerger.cc | 3 + 6 files changed, 281 insertions(+), 15 deletions(-) diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 4906a90a7..af8a8429d 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -124,6 +124,8 @@ class CzarConfig { /// Getters for result aggregation options. int getMaxTableSizeMB() const { return _maxTableSizeMB->getVal(); } int getMaxSqlConnectionAttempts() const { return _maxSqlConnectionAttempts->getVal(); } + unsigned int getMaxTransferMemMB() const { return _resultMaxTransferMemMB->getVal(); } + std::string getTransferMethod() const { return _resultTransferMethod->getVal(); } /// The size of the TCP connection pool within the client API that is used /// by the merger to pool result files from workers via the HTTP protocol. @@ -288,7 +290,11 @@ class CzarConfig { util::ConfigValTInt::create(_configValMap, "resultdb", "maxhttpconnections", notReq, 2000); CVTIntPtr _oldestResultKeptDays = util::ConfigValTInt::create(_configValMap, "resultdb", "oldestResultKeptDays", notReq, 30); - + // This must be larger than _maxTableSizeMB + CVTUIntPtr _resultMaxTransferMemMB = + util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); + CVTStrPtr _resultTransferMethod = + util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memory"); /// Get all the elements in the css section. CVTStrPtr _cssTechnology = util::ConfigValTStr::create(_configValMap, "css", "technology", notReq, "mysql"); diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index bbf2e1397..ec66c0a84 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -200,7 +200,22 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // Note the fixed capacity of the stream which allows up to 2 records to be buffered // in the stream. This is enough to hide the latency of the HTTP connection and // the time needed to read the file. - auto csvStream = mysql::CsvStream::create(2); + auto transferMethod = mysql::MemoryTracker::get()->getTransferMethod(); + mysql::CsvStream::Ptr csvStream; + switch (transferMethod) { + case mysql::MemoryTracker::MEMORY: + // This version should keep the result table valid even if there's + // a worker failure mid data transmission. + csvStream = mysql::CsvStrMem::create(fileSize); + break; + case mysql::MemoryTracker::STREAM: + // This may contaminate the result table if a worker fails. + csvStream = mysql::CsvStream::create(2); + break; + default: + throw util::Bug(ERR_LOC, + "MergingHandler::_mergeHttp UNKNOWN merge method " + to_string(transferMethod)); + } _csvStream = csvStream; // This must be after setting _csvStream to avoid cancelFileMerge() @@ -211,7 +226,7 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } string fileReadErrorMsg; - thread csvThread([uberJob, csvStream, fileUrl, fileSize, &fileReadErrorMsg]() { + auto csvLambda = [uberJob, csvStream, fileUrl, fileSize, &fileReadErrorMsg]() { size_t bytesRead = 0; fileReadErrorMsg = ::readHttpFileAndMerge( uberJob, fileUrl, fileSize, @@ -235,7 +250,8 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber if (!fileReadErrorMsg.empty()) { csvStream->push(nullptr, 0); } - }); + }; + csvStream->setLambda(csvLambda); // Attempt the actual merge. bool fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvStream); @@ -250,7 +266,7 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber _setError(ccontrol::MSG_RESULT_ERROR, "merge stream contaminated", util::ErrorCode::RESULT_IMPORT); } - csvThread.join(); + csvStream->join(); if (!fileReadErrorMsg.empty()) { LOGS(_log, LOG_LVL_WARN, __func__ << " result file read failed"); _setError(ccontrol::MSG_HTTP_RESULT, fileReadErrorMsg, util::ErrorCode::RESULT_IMPORT); @@ -263,8 +279,7 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // is finished. If any bytes were written, the result table is ruined. mergeEStatus.contaminated = csvStream->getBytesWritten() > 0; } - // TODO:UJ Make it impossible to contaminate the result table for all errors - // short of czar or mariadb crash. + return mergeEStatus; } diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 52b1bafe9..3f2ad5abd 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -54,6 +54,7 @@ #include "http/ClientConnPool.h" #include "http/MetaModule.h" #include "http/Method.h" +#include "mysql/CsvBuffer.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qproc/DatabaseModels.h" @@ -171,6 +172,19 @@ Czar::Czar(string const& configFilePath, string const& czarName) const int year = 60 * 60 * 24 * 365; _idCounter = uint64_t(tv.tv_sec % year) * 1000 + tv.tv_usec / 1000; + size_t const MB_SIZE_BYTES = 1024 * 1024; + size_t maxResultTableSizeBytes = _czarConfig->getMaxTableSizeMB() * MB_SIZE_BYTES; + + size_t maxMemToUse = _czarConfig->getMaxTransferMemMB() * MB_SIZE_BYTES; + if (maxMemToUse < maxResultTableSizeBytes) { + throw util::Bug(ERR_LOC, "Configuration error resultdb maxTransferMemMB=" + + to_string(_czarConfig->getMaxTransferMemMB()) + + " must be larger than maxtablesize_mb= " + + to_string(_czarConfig->getMaxTableSizeMB())); + } + string const transferMethod = _czarConfig->getTransferMethod(); + mysql::MemoryTracker::setup(transferMethod, maxMemToUse); + auto databaseModels = qproc::DatabaseModels::create(_czarConfig->getCssConfigMap(), _czarConfig->getMySqlResultConfig()); diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index 7113b63ff..10c461b28 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -27,6 +27,7 @@ // System headers #include #include +#include #include #include @@ -39,9 +40,12 @@ // Qserv headers #include "mysql/LocalInfileError.h" #include "mysql/MySqlUtils.h" +#include "util/Bug.h" namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.mysql.CsvBuffer"); + std::string const mysqlNull("\\N"); int const largeRowThreshold = 500 * 1024; // should be less than 0.5 * infileBufferSize @@ -307,6 +311,13 @@ bool CsvStream::empty() const { return _records.empty(); } +void CsvStream::waitReadyToRead() { + // No need to wait for this class + std::thread thrd(_csvLambda); + _thrd = std::move(thrd); + _thrdStarted = true; +} + /** * CsvStreamBuffer is a CsvBuffer that reads from a CsvStream. It is used to read * data from a CsvStream in a buffered manner. @@ -355,4 +366,90 @@ std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& return std::make_shared(csvStream); } +MemoryTracker::Ptr MemoryTracker::_globalMt; + +void MemoryTracker::setup(std::string const& transferMethodStr, size_t max) { + if (_globalMt != nullptr) { + throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); + } + TransferMethod tm = transferMethodFromString(transferMethodStr); + _globalMt = MemoryTracker::Ptr(new MemoryTracker(tm, max)); +} + +MemoryTracker::TransferMethod MemoryTracker::transferMethodFromString(std::string const& strType) { + std::string str; + for (unsigned char c : strType) { + str += std::tolower(c); + } + TransferMethod tMethod; + if (str == "memory") { + tMethod = MEMORY; + LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to memory"); + } else if (str == "stream") { + tMethod = STREAM; + LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to stream"); + } else { + tMethod = MEMORY; + LOGS(_log, LOG_LVL_ERROR, + "Result TransferMethod set to memory due to invalid string '" << strType << "'"); + } + return tMethod; +} + +MemoryTracker::MemoryRaii::Ptr MemoryTracker::createRaii(size_t fileSize) { + if (fileSize > _max) { + throw util::Bug(ERR_LOC, "MemoryTracker::createRaii file too large " + std::to_string(fileSize) + + " max=" + std::to_string(_max)); + } + std::unique_lock ulck(_mtx); + _cv.wait(ulck, [this, fileSize]() { return (fileSize + _total) < _max; }); + MemoryRaii::Ptr pRaii(new MemoryRaii(fileSize)); + return pRaii; +} + +void MemoryTracker::_incrTotal(size_t sz) { + // _mtx must already be locked. + _total += sz; + _cv.notify_one(); // Many items may be waiting on a large file, so there may be room for more +} + +void MemoryTracker::_decrTotal(size_t sz) { + std::lock_guard ulck(_mtx); + if (sz > _total) { + throw util::Bug(ERR_LOC, "MemoryTracker::_decrTotal sz=" + std::to_string(sz) + + " > total=" + std::to_string(_total)); + } + _total -= sz; + _cv.notify_one(); +} + +void CsvStrMem::waitReadyToRead() { + auto memTrack = MemoryTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMem::waitReadyToRead MemoryTracker is NULL"); + } + _memRaii = memTrack->createRaii(_expectedBytes); + + // Read directly without starting a separate thread. + _csvLambda(); +} + +void CsvStrMem::push(char const* data, std::size_t size) { + _bytesRead += size; + // Push is always ok, no need to wait. + if (_cancelled) return; + if (data != nullptr && size != 0) { + _records.emplace_back(std::make_shared(data, size)); + } else { + // Empty string is meant to indicate the end of the stream. + _records.emplace_back(std::make_shared()); + } +} + +std::shared_ptr CsvStrMem::pop() { + std::shared_ptr front = _records.front(); + _records.pop_front(); + return front; +} + } // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index e9abcd113..705cf2062 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -26,10 +26,12 @@ // System headers #include #include +#include #include #include #include #include +#include // Third-party headers #include @@ -71,6 +73,8 @@ std::shared_ptr newResCsvBuffer(MYSQL_RES* result); */ class CsvStream { public: + using Ptr = std::shared_ptr; + /** * Factory function to create a new CsvStream object. * @param maxRecords The maximum number of records in the stream @@ -84,7 +88,7 @@ class CsvStream { CsvStream() = delete; CsvStream(CsvStream const&) = delete; CsvStream& operator=(CsvStream const&) = delete; - ~CsvStream() = default; + virtual ~CsvStream() = default; /** * Push a new record to the stream. The record is a string of bytes. @@ -95,7 +99,7 @@ class CsvStream { * @param data The record to be pushed to the stream * @param size The size of the record */ - void push(char const* data, std::size_t size); + virtual void push(char const* data, std::size_t size); /** * Call to break push operations if the results are no longer needed. @@ -106,14 +110,14 @@ class CsvStream { * to a worker is lost during file transfer. * @see UberJob::killUberJob() */ - void cancel(); + virtual void cancel(); /** * Pop a record from the stream. The method will block if the stream is empty * until a record is pushed. * @return A shared pointer to the popped record or an empty string for the end of the stream */ - std::shared_ptr pop(); + virtual std::shared_ptr pop(); /** * Check if the stream is empty. @@ -133,16 +137,33 @@ class CsvStream { */ bool getContaminated() const { return _contaminated; } -private: + /// The function to run to read/push the data from the worker. + void setLambda(std::function csvLambda) { _csvLambda = csvLambda; } + + /// In this class, no waiting, just start the read/push thread. + virtual void waitReadyToRead(); + + /// Join the thread, must be called from the same thread that called waitReadyToRead + virtual void join() { + if (_thrdStarted) _thrd.join(); + } + +protected: CsvStream(std::size_t maxRecords); + std::function _csvLambda; + bool _cancelled = false; + std::atomic _bytesWritten; + std::list> _records; + +private: mutable std::mutex _mtx; std::condition_variable _cv; std::size_t const _maxRecords; - std::list> _records; - std::atomic _bytesWritten; - bool _cancelled = false; + std::atomic _contaminated = false; + std::thread _thrd; + bool _thrdStarted = false; }; /** @@ -153,5 +174,115 @@ class CsvStream { */ std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& csvStream); +/// Track how much space is needed to store the current UberJob results while +/// transferring them from the workers and merging them to the result table. +/// How this effects the process depends on the TransferMethod. RAII methods +/// are used to ensure all allocations are freed. +/// STREAM is unaffected by the amount of memory used, but may contaminate +/// the final result table on worker failure, forcing the user query +/// to be cancelled. Not much of an issue with small numbers of workers. +/// MEMORY UberJob results are stored in memory until all data is retrieved +/// from the worker and then merged to the result table. This isolates +/// worker failures from contaminating the result table. The down side +/// is that The result transfer won't be started until the `_total` +/// including the new data is less than the `_max`. It is terribly +/// important that `_max` is greater than the maximum result size +/// (currently 5GB). This should be checked when calling +/// MemoryTracker::setup. +/// MEMORYDISK TODO:UJ - Instead new transfers waiting for memory to be +/// freed, most of the data will be written to disk when `_max` is +/// reached. The current plan is, per UberJob, to write create a +/// few CsvBuffers as is done now, and then write everything to +/// disk, and have pop read off disk when it runs out of existing +/// CsvBuffers. UberJobs with reasonable result sizes should be +/// unaffected. +/// DISK TODO:UJ - Write everything to disk, have pop read the file +/// and use it to create CsvBuffers. (Maybe MEMORYDISK with +/// a _max of zero would be enough). +/// TODO:UJ - Test the different options and reorganize code. +class MemoryTracker { +public: + using Ptr = std::shared_ptr; + + MemoryTracker() = delete; + + enum TransferMethod { STREAM, MEMORY }; + static TransferMethod transferMethodFromString(std::string const& str); + + /// Return the TransferMethod. + /// @see MemoryTracker + TransferMethod getTransferMethod() const { return _transferMethod; } + + /// This class makes certain that any memory added to MemoryTracker + /// is removed from MemoryTracker. + class MemoryRaii { + public: + using Ptr = std::shared_ptr; + MemoryRaii() = delete; + ~MemoryRaii() { _globalMt->_decrTotal(memSize); } + + size_t const memSize; + friend class MemoryTracker; + + private: + /// Only to be called by createRaii(), which locks the mutex. + explicit MemoryRaii(size_t memSize_) : memSize(memSize_) { _globalMt->_incrTotal(memSize); } + }; + friend class MemoryRaii; + + static void setup(std::string const& transferMethod, size_t max); + static Ptr get() { return _globalMt; } + + MemoryRaii::Ptr createRaii(size_t fileSize); + +private: + explicit MemoryTracker(TransferMethod transferMethod, size_t max) + : _transferMethod(transferMethod), _max(max) {} + + /// This function only to be called via createRaii. + void _incrTotal(size_t sz); + + /// This function only to be called by ~MemoryRaii() + void _decrTotal(size_t sz); + + static Ptr _globalMt; + + mutable std::mutex _mtx; + std::condition_variable _cv; + size_t _total = 0; + TransferMethod const _transferMethod; + size_t const _max; +}; + +class CsvStrMem : public CsvStream { +public: + static std::shared_ptr create(std::size_t expectedBytes) { + return std::shared_ptr(new CsvStrMem(expectedBytes)); + } + + CsvStrMem() = delete; + CsvStrMem(CsvStrMem const&) = delete; + CsvStream& operator=(CsvStream const&) = delete; + ~CsvStrMem() override = default; + + void push(char const* data, std::size_t size) override; + + std::shared_ptr pop() override; + + /// Wait if there isn't enough memory available. + void waitReadyToRead() override; + + /// No thread to join. + void join() override {}; + +private: + CsvStrMem(std::size_t expectedBytes) : CsvStream(expectedBytes + 1), _expectedBytes(expectedBytes) {}; + + std::atomic _bytesRead{0}; + size_t const _expectedBytes; + + MemoryTracker::MemoryRaii::Ptr _memRaii; +}; + } // namespace lsst::qserv::mysql #endif // LSST_QSERV_MYSQL_CSVBUFFER_H diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index d05ad55cc..3e7b7bbfd 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -198,6 +198,9 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi }; auto tct = make_shared>(cbf); + // Start downloading. + csvStream->waitReadyToRead(); + bool ret = false; // Add columns to rows in virtFile. util::Timer virtFileT; From ef786f1396aed36a8ef9335e5340332668837975 Mon Sep 17 00:00:00 2001 From: John Gates Date: Fri, 20 Jun 2025 16:00:21 -0700 Subject: [PATCH 02/11] Added memory/disk hybrid for transfering csv files. --- src/cconfig/CzarConfig.h | 3 +- src/czar/Czar.cc | 3 +- src/mysql/CsvBuffer.cc | 237 +++++++++++++++++++++++++++++--------- src/mysql/CsvBuffer.h | 97 ++++++++++++++-- src/rproc/InfileMerger.cc | 12 +- 5 files changed, 285 insertions(+), 67 deletions(-) diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index af8a8429d..d831108ee 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -125,6 +125,7 @@ class CzarConfig { int getMaxTableSizeMB() const { return _maxTableSizeMB->getVal(); } int getMaxSqlConnectionAttempts() const { return _maxSqlConnectionAttempts->getVal(); } unsigned int getMaxTransferMemMB() const { return _resultMaxTransferMemMB->getVal(); } + /// May be "stream", "memory", or "memdisk" std::string getTransferMethod() const { return _resultTransferMethod->getVal(); } /// The size of the TCP connection pool within the client API that is used @@ -294,7 +295,7 @@ class CzarConfig { CVTUIntPtr _resultMaxTransferMemMB = util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); CVTStrPtr _resultTransferMethod = - util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memory"); + util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memdisk"); /// Get all the elements in the css section. CVTStrPtr _cssTechnology = util::ConfigValTStr::create(_configValMap, "css", "technology", notReq, "mysql"); diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 3f2ad5abd..d0c571de0 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -183,7 +183,8 @@ Czar::Czar(string const& configFilePath, string const& czarName) to_string(_czarConfig->getMaxTableSizeMB())); } string const transferMethod = _czarConfig->getTransferMethod(); - mysql::MemoryTracker::setup(transferMethod, maxMemToUse); + string const transferDirectory = "/tmp"; //&&& use config + mysql::MemoryTracker::setup(transferMethod, maxMemToUse, transferDirectory); auto databaseModels = qproc::DatabaseModels::create(_czarConfig->getCssConfigMap(), _czarConfig->getMySqlResultConfig()); diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index 10c461b28..c3eaf01d3 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -42,11 +42,14 @@ #include "mysql/MySqlUtils.h" #include "util/Bug.h" +using namespace std; +namespace sfs = std::filesystem; + namespace { LOG_LOGGER _log = LOG_GET("lsst.qserv.mysql.CsvBuffer"); -std::string const mysqlNull("\\N"); +string const mysqlNull("\\N"); int const largeRowThreshold = 500 * 1024; // should be less than 0.5 * infileBufferSize } // namespace @@ -80,13 +83,13 @@ inline unsigned updateEstRowSize(unsigned lastRowSize, Row const& r) { return lastRowSize < rowSize ? rowSize : lastRowSize; } -inline int addString(char* cursor, std::string const& s) { +inline int addString(char* cursor, string const& s) { int const sSize = s.size(); memcpy(cursor, s.data(), sSize); return sSize; } -inline int maxColFootprint(int columnLength, std::string const& sep) { +inline int maxColFootprint(int columnLength, string const& sep) { const int overhead = 2 + sep.size(); // NULL decl + sep size return overhead + (2 * columnLength); } @@ -112,7 +115,7 @@ class ResCsvBuffer : public CsvBuffer { bool _fetchRow(Row& r); unsigned _fetchFromLargeRow(char* buffer, int bufLen); void _initializeLargeRow(Row const& largeRow); - std::string dump() const override; + string dump() const override; private: MYSQL_RES* _result; @@ -123,8 +126,8 @@ class ResCsvBuffer : public CsvBuffer { Row _largeRow; int _fieldOffset; - std::string _sep; - std::string _rowSep; + string _sep; + string _rowSep; }; ResCsvBuffer::ResCsvBuffer(MYSQL_RES* result) @@ -135,8 +138,8 @@ ResCsvBuffer::ResCsvBuffer(MYSQL_RES* result) // cout << _numFields << " fields per row\n"; } -std::string ResCsvBuffer::dump() const { - std::string str = std::string("ResCsvBuffer _numFields=") + std::to_string(_numFields); +string ResCsvBuffer::dump() const { + string str = string("ResCsvBuffer _numFields=") + to_string(_numFields); return str; } @@ -254,38 +257,36 @@ void ResCsvBuffer::_initializeLargeRow(Row const& largeRow) { _fieldOffset = 0; } -std::shared_ptr newResCsvBuffer(MYSQL_RES* result) { - return std::make_shared(result); -} +shared_ptr newResCsvBuffer(MYSQL_RES* result) { return make_shared(result); } -CsvStream::CsvStream(std::size_t maxRecords) : _maxRecords(maxRecords) { +CsvStream::CsvStream(size_t maxRecords) : _maxRecords(maxRecords) { if (maxRecords == 0) { - throw std::invalid_argument("CsvStream::CsvStream: maxRecords must be greater than 0"); + throw invalid_argument("CsvStream::CsvStream: maxRecords must be greater than 0"); } } void CsvStream::cancel() { - std::unique_lock lock(_mtx); + unique_lock lock(_mtx); _cancelled = true; _cv.notify_all(); } -void CsvStream::push(char const* data, std::size_t size) { - std::unique_lock lock(_mtx); +void CsvStream::push(char const* data, size_t size) { + unique_lock lock(_mtx); _cv.wait(lock, [this]() { return (_records.size() < _maxRecords) || _cancelled; }); if (_cancelled) return; if (data != nullptr && size != 0) { - _records.emplace_back(std::make_shared(data, size)); + _records.emplace_back(make_shared(data, size)); } else { // Empty string is meant to indicate the end of the stream. - _records.emplace_back(std::make_shared()); + _records.emplace_back(make_shared()); } _cv.notify_one(); } -std::shared_ptr CsvStream::pop() { - std::unique_lock lock(_mtx); +shared_ptr CsvStream::pop() { + unique_lock lock(_mtx); _cv.wait(lock, [this]() { return (!_records.empty() || _cancelled); }); if (_records.empty()) { @@ -296,25 +297,25 @@ std::shared_ptr CsvStream::pop() { // database stops asking for them. // See CsvStream::cancel() _contaminated = true; - auto pstr = std::make_shared("$"); + auto pstr = make_shared("$"); _cv.notify_one(); return pstr; } - std::shared_ptr front = _records.front(); + shared_ptr front = _records.front(); _records.pop_front(); _cv.notify_one(); return front; } bool CsvStream::empty() const { - std::unique_lock lock(_mtx); + unique_lock lock(_mtx); return _records.empty(); } void CsvStream::waitReadyToRead() { // No need to wait for this class - std::thread thrd(_csvLambda); - _thrd = std::move(thrd); + thread thrd(_csvLambda); + _thrd = move(thrd); _thrdStarted = true; } @@ -329,7 +330,7 @@ void CsvStream::waitReadyToRead() { */ class CsvStreamBuffer : public CsvBuffer { public: - explicit CsvStreamBuffer(std::shared_ptr const& csvStream) : _csvStream(csvStream) {} + explicit CsvStreamBuffer(shared_ptr const& csvStream) : _csvStream(csvStream) {} ~CsvStreamBuffer() override = default; @@ -347,39 +348,53 @@ class CsvStreamBuffer : public CsvBuffer { _offset = 0; if (_str->empty()) return 0; } - unsigned const bytesToCopy = std::min(bufLen, static_cast(_str->size() - _offset)); + unsigned const bytesToCopy = min(bufLen, static_cast(_str->size() - _offset)); ::memcpy(buffer, _str->data() + _offset, bytesToCopy); _offset += bytesToCopy; _csvStream->increaseBytesWrittenBy(bytesToCopy); return bytesToCopy; } - std::string dump() const override { return "CsvStreamBuffer"; } + string dump() const override { return "CsvStreamBuffer"; } private: - std::shared_ptr _csvStream; - std::shared_ptr _str; - std::size_t _offset = 0; + shared_ptr _csvStream; + shared_ptr _str; + size_t _offset = 0; }; -std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& csvStream) { - return std::make_shared(csvStream); +shared_ptr newCsvStreamBuffer(shared_ptr const& csvStream) { + return make_shared(csvStream); } MemoryTracker::Ptr MemoryTracker::_globalMt; -void MemoryTracker::setup(std::string const& transferMethodStr, size_t max) { +void MemoryTracker::setup(string const& transferMethodStr, size_t max, string const& directory) { if (_globalMt != nullptr) { throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); } TransferMethod tm = transferMethodFromString(transferMethodStr); - _globalMt = MemoryTracker::Ptr(new MemoryTracker(tm, max)); + if (tm == MEMDISK) { + if (!verifyDir(directory)) { + throw util::Bug(ERR_LOC, "MemoryTracker::setup called with bad directory! " + directory); + } + } + _globalMt = MemoryTracker::Ptr(new MemoryTracker(tm, max, directory)); +} + +bool MemoryTracker::verifyDir(string const& dirName) { + sfs::path dir = dirName; + if (!(sfs::exists(dir) && sfs::is_directory(dir))) { + LOGS(_log, LOG_LVL_ERROR, "verifyDir, " + dirName + " is not a valid directory"); + return false; + } + return true; } -MemoryTracker::TransferMethod MemoryTracker::transferMethodFromString(std::string const& strType) { - std::string str; +MemoryTracker::TransferMethod MemoryTracker::transferMethodFromString(string const& strType) { + string str; for (unsigned char c : strType) { - str += std::tolower(c); + str += tolower(c); } TransferMethod tMethod; if (str == "memory") { @@ -388,21 +403,28 @@ MemoryTracker::TransferMethod MemoryTracker::transferMethodFromString(std::strin } else if (str == "stream") { tMethod = STREAM; LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to stream"); + } else if (str == "memdisk") { + tMethod = MEMDISK; + LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to memdisk"); } else { tMethod = MEMORY; LOGS(_log, LOG_LVL_ERROR, - "Result TransferMethod set to memory due to invalid string '" << strType << "'"); + "Result TransferMethod set to memory due to invalid string '" + << strType << "'" + << " valid entries are 'memory', 'stream', 'memdisk'"); } return tMethod; } -MemoryTracker::MemoryRaii::Ptr MemoryTracker::createRaii(size_t fileSize) { +MemoryTracker::MemoryRaii::Ptr MemoryTracker::createRaii(size_t fileSize, bool wait) { if (fileSize > _max) { - throw util::Bug(ERR_LOC, "MemoryTracker::createRaii file too large " + std::to_string(fileSize) + - " max=" + std::to_string(_max)); + throw util::Bug(ERR_LOC, "MemoryTracker::createRaii file too large " + to_string(fileSize) + + " max=" + to_string(_max)); + } + unique_lock ulck(_mtx); + if (wait) { + _cv.wait(ulck, [this, fileSize]() { return (fileSize + _total) < _max; }); } - std::unique_lock ulck(_mtx); - _cv.wait(ulck, [this, fileSize]() { return (fileSize + _total) < _max; }); MemoryRaii::Ptr pRaii(new MemoryRaii(fileSize)); return pRaii; } @@ -414,10 +436,10 @@ void MemoryTracker::_incrTotal(size_t sz) { } void MemoryTracker::_decrTotal(size_t sz) { - std::lock_guard ulck(_mtx); + lock_guard ulck(_mtx); if (sz > _total) { - throw util::Bug(ERR_LOC, "MemoryTracker::_decrTotal sz=" + std::to_string(sz) + - " > total=" + std::to_string(_total)); + throw util::Bug(ERR_LOC, + "MemoryTracker::_decrTotal sz=" + to_string(sz) + " > total=" + to_string(_total)); } _total -= sz; _cv.notify_one(); @@ -428,28 +450,135 @@ void CsvStrMem::waitReadyToRead() { if (memTrack == nullptr) { throw util::Bug(ERR_LOC, "CsvStrMem::waitReadyToRead MemoryTracker is NULL"); } - _memRaii = memTrack->createRaii(_expectedBytes); + bool const wait = true; + _memRaii = memTrack->createRaii(_expectedBytes, wait); // Read directly without starting a separate thread. _csvLambda(); } -void CsvStrMem::push(char const* data, std::size_t size) { - _bytesRead += size; +void CsvStrMem::push(char const* data, size_t size) { // Push is always ok, no need to wait. if (_cancelled) return; + _bytesRead += size; if (data != nullptr && size != 0) { - _records.emplace_back(std::make_shared(data, size)); + _records.emplace_back(make_shared(data, size)); } else { // Empty string is meant to indicate the end of the stream. - _records.emplace_back(std::make_shared()); + _records.emplace_back(make_shared()); } } -std::shared_ptr CsvStrMem::pop() { - std::shared_ptr front = _records.front(); +shared_ptr CsvStrMem::pop() { + shared_ptr front = _records.front(); _records.pop_front(); return front; } +CsvStrMemDisk::CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId) + : CsvStrMem(expectedBytes), _qId(qId), _ujId(ujId) { + auto memTrack = MemoryTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMemDisk constructor MemoryTracker is NULL"); + } + sfs::path fPath = memTrack->getDirectory(); + string fileName = memTrack->getBaseFileName() + "_" + to_string(_qId) + "_" + to_string(ujId); + fPath /= fileName; + _filePath = fPath; +} + +void CsvStrMemDisk::waitReadyToRead() { + auto memTrack = MemoryTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMemDisk::waitReadyToRead MemoryTracker is NULL"); + } + bool const nowait = false; + _memRaii = memTrack->createRaii(_expectedBytes, nowait); + + // Read directly without starting a separate thread. + _csvLambda(); +} + +bool CsvStrMemDisk::_mustWriteToFile() { + // Once writing to file, this instance must keep writing to file. + if (_writingToFile) return true; + + auto memTrack = MemoryTracker::get(); + // If too much memory is being used for transfers, star writing large transfers to files. + if (memTrack->getTotal() > memTrack->getMax()) { + if (_records.size() > _minRecordsSize && _bytesRead > _minBytesInMem) { + _writingToFile = true; + } + } + return _writingToFile; +} + +void CsvStrMemDisk::push(char const* data, size_t size) { + // Push is always ok, no need to wait. + if (_cancelled) return; + _bytesRead += size; + if (data != nullptr && size != 0) { + if (_mustWriteToFile()) { + _writeTofile(data, size); + return; + } + _records.emplace_back(make_shared(data, size)); + } else { + // Empty string is meant to indicate the end of the stream. + _records.emplace_back(make_shared()); + } +} + +shared_ptr CsvStrMemDisk::pop() { + if (_records.size() > 0) { + shared_ptr front = _records.front(); + _records.pop_front(); + return front; + } + return _readFromFile(); +} + +void CsvStrMemDisk::_writeTofile(char const* data, std::size_t size) { + // Open the file if needed + auto oldState = _fState.exchange(OPEN_W); + if (oldState == INIT) { + _file.open(_filePath, fstream::out); + } + if (!_file.is_open() || _fState != OPEN_W) { + LOGS(_log, LOG_LVL_ERROR, + "CsvStrMemDisk::_writeTofile file isn't open " << _filePath << " or bad state=" << _fState); + _fileError = true; + return; + } + + _file.write(data, size); +} + +std::shared_ptr CsvStrMemDisk::_readFromFile() { + if (_fState == OPEN_W) { + _fState = CLOSE_W; + _file.close(); + } + auto oldState = _fState.exchange(OPEN_R); + if (oldState == CLOSE_W) { + _file.open(_filePath, fstream::in); + _bytesLeft = _bytesWritten; + } + if (!_file.is_open() || _fState != OPEN_R) { + // This is extremely unlikely + if (!getContaminated()) + LOGS(_log, LOG_LVL_ERROR, + "CsvStrMemDisk::_readFromfile file isn't open " << _filePath << " or bad state=" << _fState); + setContaminated(); + return make_shared("$"); + } + + std::size_t buffSz = std::min(10'000'000ul, _bytesLeft); + auto strPtr = make_shared(); + strPtr->resize(buffSz); + _file.read(strPtr->data(), buffSz); + _bytesLeft -= buffSz; + return strPtr; +} + } // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index 705cf2062..cd3cb23d1 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -24,8 +24,10 @@ #define LSST_QSERV_MYSQL_CSVBUFFER_H // System headers +#include #include #include +#include #include #include #include @@ -36,6 +38,9 @@ // Third-party headers #include +// qserv headers +#include "global/intTypes.h" + namespace lsst::qserv::mysql { /** @@ -112,6 +117,11 @@ class CsvStream { */ virtual void cancel(); + /* + * Return true if this instance has been cancelled. + */ + bool isCancelled() const { return _cancelled; } + /** * Pop a record from the stream. The method will block if the stream is empty * until a record is pushed. @@ -151,6 +161,8 @@ class CsvStream { protected: CsvStream(std::size_t maxRecords); + void setContaminated() { _contaminated = true; } + std::function _csvLambda; bool _cancelled = false; std::atomic _bytesWritten; @@ -196,9 +208,6 @@ std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& /// disk, and have pop read off disk when it runs out of existing /// CsvBuffers. UberJobs with reasonable result sizes should be /// unaffected. -/// DISK TODO:UJ - Write everything to disk, have pop read the file -/// and use it to create CsvBuffers. (Maybe MEMORYDISK with -/// a _max of zero would be enough). /// TODO:UJ - Test the different options and reorganize code. class MemoryTracker { public: @@ -206,8 +215,10 @@ class MemoryTracker { MemoryTracker() = delete; - enum TransferMethod { STREAM, MEMORY }; + enum TransferMethod { STREAM, MEMORY, MEMDISK }; static TransferMethod transferMethodFromString(std::string const& str); + static bool verifyDir(std::string const& dirName); + static std::string getBaseFileName() { return std::string("qservtransfer"); } /// Return the TransferMethod. /// @see MemoryTracker @@ -230,14 +241,24 @@ class MemoryTracker { }; friend class MemoryRaii; - static void setup(std::string const& transferMethod, size_t max); + static void setup(std::string const& transferMethod, size_t max, std::string const& directory); static Ptr get() { return _globalMt; } - MemoryRaii::Ptr createRaii(size_t fileSize); + /// Create a MemoryRaii instance to track `fileSize` bytes, and wait for free memory if `wait` is true. + MemoryRaii::Ptr createRaii(size_t fileSize, bool wait); + + size_t getTotal() const { + std::lock_guard lg(_mtx); + return _total; + } + + size_t getMax() const { return _max; } + + std::string getDirectory() const { return _directory; } private: - explicit MemoryTracker(TransferMethod transferMethod, size_t max) - : _transferMethod(transferMethod), _max(max) {} + explicit MemoryTracker(TransferMethod transferMethod, size_t max, std::string const& directory) + : _transferMethod(transferMethod), _max(max), _directory(directory) {} /// This function only to be called via createRaii. void _incrTotal(size_t sz); @@ -252,6 +273,7 @@ class MemoryTracker { size_t _total = 0; TransferMethod const _transferMethod; size_t const _max; + std::string const _directory; }; class CsvStrMem : public CsvStream { @@ -275,7 +297,7 @@ class CsvStrMem : public CsvStream { /// No thread to join. void join() override {}; -private: +protected: CsvStrMem(std::size_t expectedBytes) : CsvStream(expectedBytes + 1), _expectedBytes(expectedBytes) {}; std::atomic _bytesRead{0}; @@ -284,5 +306,62 @@ class CsvStrMem : public CsvStream { MemoryTracker::MemoryRaii::Ptr _memRaii; }; +/// Store transfer data in memory until too much memory is being used. +/// By setting the maximum acceptable amount of memory to 0, this +/// effectively becomes writing results to disk. +/// Collecting data from the worker, writing it to disk, reading +/// it back, and merging is expected to be linear, run within a +/// single thread. +class CsvStrMemDisk : public CsvStrMem { +public: + enum FileState { INIT, OPEN_W, CLOSE_W, OPEN_R, CLOSED }; + + static std::shared_ptr create(std::size_t expectedBytes, QueryId qId, UberJobId ujId) { + return std::shared_ptr(new CsvStrMemDisk(expectedBytes, qId, ujId)); + } + + CsvStrMemDisk() = delete; + CsvStrMemDisk(CsvStrMemDisk const&) = delete; + CsvStrMemDisk& operator=(CsvStrMemDisk const&) = delete; + ~CsvStrMemDisk() override = default; + + void push(char const* data, std::size_t size) override; + + std::shared_ptr pop() override; + + /// This version never waits. + void waitReadyToRead() override; + + /// True if a file error happened before results would be contaminated. + bool isFileError() const { return _fileError; } + +private: + CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId); + + void _writeTofile(char const* data, std::size_t size); + + /// Read from the file, which should only happen after all writing has finished. + std::shared_ptr _readFromFile(); + + bool _mustWriteToFile(); + + /// Have at least on record ready to be pushed + unsigned int const _minRecordsSize = 1; + size_t const _minBytesInMem = 10'000'000; // &&& config + + bool _writingToFile = false; + std::string const _directory; + std::string const _baseName; + QueryId const _qId; + UberJobId const _ujId; + + std::atomic _fState = INIT; + std::string _filePath; ///< file path, constant once set. + std::fstream _file; + + std::atomic _fileError = false; + size_t _bytesLeft = 0; +}; + } // namespace lsst::qserv::mysql #endif // LSST_QSERV_MYSQL_CSVBUFFER_H diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 3e7b7bbfd..c027ebecf 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -222,12 +222,20 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi // Don't merge if the query got cancelled. auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { - csvStream->cancel(); // After this point, the file has to be read + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete() || + csvStream->isCancelled()) { + csvStream->cancel(); return true; } + auto csvStrMemDisk = std::dynamic_pointer_cast(csvStream); + if (csvStrMemDisk != nullptr && csvStrMemDisk->isFileError()) { + csvStrMemDisk->cancel(); + return false; + } + auto start = std::chrono::system_clock::now(); + // The following will call some version of CsvStream::pop() at least once. ret = _applyMysqlMyIsam(infileStatement, fileSize); auto end = std::chrono::system_clock::now(); auto mergeDur = std::chrono::duration_cast(end - start); From 2b51c1e404bbac55cdc8794b2e9afa3e7a32ccbc Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 25 Jun 2025 14:42:37 -0700 Subject: [PATCH 03/11] Some integration tests failing. --- src/cconfig/CzarConfig.h | 15 ++++++- src/ccontrol/MergingHandler.cc | 19 ++++++--- src/czar/ActiveWorker.cc | 3 +- src/czar/Czar.cc | 12 ++---- src/czar/CzarChunkMap.cc | 8 +++- src/czar/CzarChunkMap.h | 2 +- src/mysql/CsvBuffer.cc | 78 ++++++++++++++++++++++++---------- src/mysql/CsvBuffer.h | 52 ++++++++++++++--------- 8 files changed, 127 insertions(+), 62 deletions(-) diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index d831108ee..38f74b5cf 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -127,6 +127,12 @@ class CzarConfig { unsigned int getMaxTransferMemMB() const { return _resultMaxTransferMemMB->getVal(); } /// May be "stream", "memory", or "memdisk" std::string getTransferMethod() const { return _resultTransferMethod->getVal(); } + /// Return the transfer directory, which defaults to /tmp, which is bad for performance. + std::string getTransferDir() const { return _resultTransferDir->getVal(); } + + /// Return the minimum amount of memory per UberJob to keep in memory. This much transfer + /// data will be stored in memory regardless of other conditions. + unsigned int getTransferMinMBInMem() const { return _resultTransferMinMBInMem->getVal(); } /// The size of the TCP connection pool within the client API that is used /// by the merger to pool result files from workers via the HTTP protocol. @@ -293,9 +299,16 @@ class CzarConfig { util::ConfigValTInt::create(_configValMap, "resultdb", "oldestResultKeptDays", notReq, 30); // This must be larger than _maxTableSizeMB CVTUIntPtr _resultMaxTransferMemMB = - util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); + //&&&util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); + util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 0); CVTStrPtr _resultTransferMethod = util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memdisk"); + CVTStrPtr _resultTransferDir = + util::ConfigValTStr::create(_configValMap, "resultdb", "transferDir", notReq, "/tmp"); + CVTUIntPtr _resultTransferMinMBInMem = + util::ConfigValTUInt::create(_configValMap, "resultdb", "transferMinMBInMem", notReq, 0); + //&&&util::ConfigValTUInt::create(_configValMap, "resultdb", "transferMinMBInMem", notReq, 10); + /// Get all the elements in the css section. CVTStrPtr _cssTechnology = util::ConfigValTStr::create(_configValMap, "css", "technology", notReq, "mysql"); diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index ec66c0a84..eabc8050a 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -200,21 +200,28 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // Note the fixed capacity of the stream which allows up to 2 records to be buffered // in the stream. This is enough to hide the latency of the HTTP connection and // the time needed to read the file. - auto transferMethod = mysql::MemoryTracker::get()->getTransferMethod(); + auto transferMethod = mysql::TransferTracker::get()->getTransferMethod(); mysql::CsvStream::Ptr csvStream; switch (transferMethod) { - case mysql::MemoryTracker::MEMORY: + case mysql::TransferTracker::MEMORY: // This version should keep the result table valid even if there's - // a worker failure mid data transmission. + // a worker failure mid data transmission, but may block + // starting transfers when too much memory is in use. csvStream = mysql::CsvStrMem::create(fileSize); break; - case mysql::MemoryTracker::STREAM: + case mysql::TransferTracker::MEMDISK: + // This version should keep the result table valid even if there's + // a worker failure mid data transmission, but may start writing + // transfer data to disk when memory is low. + csvStream = mysql::CsvStrMemDisk::create(fileSize, uberJob->getQueryId(), uberJob->getUjId()); + break; + case mysql::TransferTracker::STREAM: // This may contaminate the result table if a worker fails. csvStream = mysql::CsvStream::create(2); break; default: - throw util::Bug(ERR_LOC, - "MergingHandler::_mergeHttp UNKNOWN merge method " + to_string(transferMethod)); + throw util::Bug(ERR_LOC, "MergingHandler::_mergeHttp UNKNOWN transferMethod " + + to_string(transferMethod)); } _csvStream = csvStream; diff --git a/src/czar/ActiveWorker.cc b/src/czar/ActiveWorker.cc index 81d011532..12aae1fce 100644 --- a/src/czar/ActiveWorker.cc +++ b/src/czar/ActiveWorker.cc @@ -288,7 +288,8 @@ void ActiveWorkerMap::updateMap(protojson::WorkerContactInfo::WCMap const& wcMap auto iter = _awMap.find(wcKey); if (iter == _awMap.end()) { auto newAW = ActiveWorker::create(wcVal, czInfo, replicationInstanceId, replicationAuthKey); - LOGS(_log, LOG_LVL_INFO, cName(__func__) << " ActiveWorker created for " << wcKey); + LOGS(_log, LOG_LVL_INFO, + cName(__func__) << " ActiveWorker created for " << wcKey << " " << newAW->dump()); _awMap[wcKey] = newAW; if (_czarCancelAfterRestart) { newAW->setCzarCancelAfterRestart(_czarCancelAfterRestartCzId, _czarCancelAfterRestartQId); diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index d0c571de0..60638c5e7 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -176,15 +176,11 @@ Czar::Czar(string const& configFilePath, string const& czarName) size_t maxResultTableSizeBytes = _czarConfig->getMaxTableSizeMB() * MB_SIZE_BYTES; size_t maxMemToUse = _czarConfig->getMaxTransferMemMB() * MB_SIZE_BYTES; - if (maxMemToUse < maxResultTableSizeBytes) { - throw util::Bug(ERR_LOC, "Configuration error resultdb maxTransferMemMB=" + - to_string(_czarConfig->getMaxTransferMemMB()) + - " must be larger than maxtablesize_mb= " + - to_string(_czarConfig->getMaxTableSizeMB())); - } string const transferMethod = _czarConfig->getTransferMethod(); - string const transferDirectory = "/tmp"; //&&& use config - mysql::MemoryTracker::setup(transferMethod, maxMemToUse, transferDirectory); + string const transferDirectory = _czarConfig->getTransferDir(); + std::size_t const transferMinBytesInMem = _czarConfig->getTransferMinMBInMem() * MB_SIZE_BYTES; + mysql::TransferTracker::setup(transferMethod, maxMemToUse, transferDirectory, transferMinBytesInMem, + maxResultTableSizeBytes); auto databaseModels = qproc::DatabaseModels::create(_czarConfig->getCssConfigMap(), _czarConfig->getMySqlResultConfig()); diff --git a/src/czar/CzarChunkMap.cc b/src/czar/CzarChunkMap.cc index 3f84b3b92..3cf204b51 100644 --- a/src/czar/CzarChunkMap.cc +++ b/src/czar/CzarChunkMap.cc @@ -139,10 +139,10 @@ void CzarChunkMap::verify(string const& familyName) const { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " family=" << familyName << " verified"); } -string CzarChunkMap::dumpChunkMap(ChunkMap const& chunkMap) { +string CzarChunkMap::dumpChunkMap() const { stringstream os; os << "ChunkMap{"; - for (auto const& [cId, cDataPtr] : chunkMap) { + for (auto const& [cId, cDataPtr] : *_chunkMap) { os << "(cId=" << cId << ":"; os << ((cDataPtr == nullptr) ? "null" : cDataPtr->dump()) << ")"; } @@ -355,6 +355,10 @@ bool CzarFamilyMap::_read() { verify(familyMapPtr); + for (auto const& [fam, ccMap] : *familyMapPtr) { + LOGS(_log, LOG_LVL_INFO, "{family=" << fam << "{" << ccMap->dumpChunkMap() << "}}"); + } + _familyMap = familyMapPtr; _lastUpdateTime = qChunkMap.updateTime; diff --git a/src/czar/CzarChunkMap.h b/src/czar/CzarChunkMap.h index f3d936532..a1b7df30a 100644 --- a/src/czar/CzarChunkMap.h +++ b/src/czar/CzarChunkMap.h @@ -205,7 +205,7 @@ class CzarChunkMap { /// @throws ChunkMapException void verify(std::string const& familyName) const; - static std::string dumpChunkMap(ChunkMap const& chunkMap); + std::string dumpChunkMap() const; static std::string dumpWorkerChunkMap(WorkerChunkMap const& wcMap); diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index c3eaf01d3..bcc3ab15d 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -367,9 +368,10 @@ shared_ptr newCsvStreamBuffer(shared_ptr const& csvStream) return make_shared(csvStream); } -MemoryTracker::Ptr MemoryTracker::_globalMt; +TransferTracker::Ptr TransferTracker::_globalMt; -void MemoryTracker::setup(string const& transferMethodStr, size_t max, string const& directory) { +void TransferTracker::setup(string const& transferMethodStr, std::size_t max, string const& directory, + std::size_t minMBInMem, std::size_t maxResultTableSizeBytes) { if (_globalMt != nullptr) { throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); } @@ -379,10 +381,17 @@ void MemoryTracker::setup(string const& transferMethodStr, size_t max, string co throw util::Bug(ERR_LOC, "MemoryTracker::setup called with bad directory! " + directory); } } - _globalMt = MemoryTracker::Ptr(new MemoryTracker(tm, max, directory)); + size_t const MB_SIZE_BYTES = 1024 * 1024; + if (tm == MEMORY && max < maxResultTableSizeBytes) { + throw util::Bug(ERR_LOC, + "Configuration error resultdb maxTransferMemMB=" + to_string(max / MB_SIZE_BYTES) + + " must be larger than maxtablesize_mb= " + + to_string(maxResultTableSizeBytes / MB_SIZE_BYTES)); + } + _globalMt = TransferTracker::Ptr(new TransferTracker(tm, max, directory, minMBInMem)); } -bool MemoryTracker::verifyDir(string const& dirName) { +bool TransferTracker::verifyDir(string const& dirName) { sfs::path dir = dirName; if (!(sfs::exists(dir) && sfs::is_directory(dir))) { LOGS(_log, LOG_LVL_ERROR, "verifyDir, " + dirName + " is not a valid directory"); @@ -391,7 +400,7 @@ bool MemoryTracker::verifyDir(string const& dirName) { return true; } -MemoryTracker::TransferMethod MemoryTracker::transferMethodFromString(string const& strType) { +TransferTracker::TransferMethod TransferTracker::transferMethodFromString(string const& strType) { string str; for (unsigned char c : strType) { str += tolower(c); @@ -416,26 +425,26 @@ MemoryTracker::TransferMethod MemoryTracker::transferMethodFromString(string con return tMethod; } -MemoryTracker::MemoryRaii::Ptr MemoryTracker::createRaii(size_t fileSize, bool wait) { - if (fileSize > _max) { - throw util::Bug(ERR_LOC, "MemoryTracker::createRaii file too large " + to_string(fileSize) + - " max=" + to_string(_max)); - } +TransferTracker::MemoryRaii::Ptr TransferTracker::createRaii(size_t fileSize, bool wait) { unique_lock ulck(_mtx); if (wait) { + if (fileSize > _max) { + throw util::Bug(ERR_LOC, "MemoryTracker::createRaii file too large " + to_string(fileSize) + + " max=" + to_string(_max)); + } _cv.wait(ulck, [this, fileSize]() { return (fileSize + _total) < _max; }); } MemoryRaii::Ptr pRaii(new MemoryRaii(fileSize)); return pRaii; } -void MemoryTracker::_incrTotal(size_t sz) { +void TransferTracker::_incrTotal(size_t sz) { // _mtx must already be locked. _total += sz; _cv.notify_one(); // Many items may be waiting on a large file, so there may be room for more } -void MemoryTracker::_decrTotal(size_t sz) { +void TransferTracker::_decrTotal(size_t sz) { lock_guard ulck(_mtx); if (sz > _total) { throw util::Bug(ERR_LOC, @@ -446,7 +455,7 @@ void MemoryTracker::_decrTotal(size_t sz) { } void CsvStrMem::waitReadyToRead() { - auto memTrack = MemoryTracker::get(); + auto memTrack = TransferTracker::get(); if (memTrack == nullptr) { throw util::Bug(ERR_LOC, "CsvStrMem::waitReadyToRead MemoryTracker is NULL"); } @@ -477,7 +486,7 @@ shared_ptr CsvStrMem::pop() { CsvStrMemDisk::CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId) : CsvStrMem(expectedBytes), _qId(qId), _ujId(ujId) { - auto memTrack = MemoryTracker::get(); + auto memTrack = TransferTracker::get(); if (memTrack == nullptr) { throw util::Bug(ERR_LOC, "CsvStrMemDisk constructor MemoryTracker is NULL"); } @@ -485,10 +494,12 @@ CsvStrMemDisk::CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId u string fileName = memTrack->getBaseFileName() + "_" + to_string(_qId) + "_" + to_string(ujId); fPath /= fileName; _filePath = fPath; + + _minBytesInMem = memTrack->getMinBytesInMem(); } void CsvStrMemDisk::waitReadyToRead() { - auto memTrack = MemoryTracker::get(); + auto memTrack = TransferTracker::get(); if (memTrack == nullptr) { throw util::Bug(ERR_LOC, "CsvStrMemDisk::waitReadyToRead MemoryTracker is NULL"); } @@ -499,11 +510,11 @@ void CsvStrMemDisk::waitReadyToRead() { _csvLambda(); } -bool CsvStrMemDisk::_mustWriteToFile() { +bool CsvStrMemDisk::_mustWriteToTmpFile() { // Once writing to file, this instance must keep writing to file. if (_writingToFile) return true; - auto memTrack = MemoryTracker::get(); + auto memTrack = TransferTracker::get(); // If too much memory is being used for transfers, star writing large transfers to files. if (memTrack->getTotal() > memTrack->getMax()) { if (_records.size() > _minRecordsSize && _bytesRead > _minBytesInMem) { @@ -518,8 +529,8 @@ void CsvStrMemDisk::push(char const* data, size_t size) { if (_cancelled) return; _bytesRead += size; if (data != nullptr && size != 0) { - if (_mustWriteToFile()) { - _writeTofile(data, size); + if (_mustWriteToTmpFile()) { + _writeToTmpfile(data, size); return; } _records.emplace_back(make_shared(data, size)); @@ -530,18 +541,21 @@ void CsvStrMemDisk::push(char const* data, size_t size) { } shared_ptr CsvStrMemDisk::pop() { + LOGS(_log, LOG_LVL_WARN, "&&& pop " << _records.size()); if (_records.size() > 0) { shared_ptr front = _records.front(); _records.pop_front(); return front; } - return _readFromFile(); + return _readFromTmpFile(); } -void CsvStrMemDisk::_writeTofile(char const* data, std::size_t size) { +void CsvStrMemDisk::_writeToTmpfile(char const* data, std::size_t size) { + LOGS(_log, LOG_LVL_WARN, "&&& _writeToTmpFile() _fState=" << _fState); // Open the file if needed auto oldState = _fState.exchange(OPEN_W); if (oldState == INIT) { + LOGS(_log, LOG_LVL_WARN, "&&& _writeToTmpFile() open out _fState=" << _fState); _file.open(_filePath, fstream::out); } if (!_file.is_open() || _fState != OPEN_W) { @@ -552,17 +566,23 @@ void CsvStrMemDisk::_writeTofile(char const* data, std::size_t size) { } _file.write(data, size); + _bytesWrittenToTmp += size; + LOGS(_log, LOG_LVL_WARN, + "&&& _writeToTmpFile() _bytesWrittenToTmp=" << _bytesWrittenToTmp << " sz=" << size); } -std::shared_ptr CsvStrMemDisk::_readFromFile() { +std::shared_ptr CsvStrMemDisk::_readFromTmpFile() { + LOGS(_log, LOG_LVL_WARN, "&&& _readFromTmpFile() _fState=" << _fState); if (_fState == OPEN_W) { + LOGS(_log, LOG_LVL_WARN, "&&& _readFromTmpFile() OPEN_W _fState=" << _fState); _fState = CLOSE_W; _file.close(); } auto oldState = _fState.exchange(OPEN_R); if (oldState == CLOSE_W) { + LOGS(_log, LOG_LVL_WARN, "&&& _readFromTmpFile() CLOSE_W _fState=" << _fState); _file.open(_filePath, fstream::in); - _bytesLeft = _bytesWritten; + _bytesLeft = _bytesWrittenToTmp; } if (!_file.is_open() || _fState != OPEN_R) { // This is extremely unlikely @@ -574,11 +594,23 @@ std::shared_ptr CsvStrMemDisk::_readFromFile() { } std::size_t buffSz = std::min(10'000'000ul, _bytesLeft); + LOGS(_log, LOG_LVL_WARN, + "&&& _readFromTmpFile() _bytesLeft=" << _bytesLeft << " buffSz=" << buffSz + << " _fState=" << _fState); auto strPtr = make_shared(); strPtr->resize(buffSz); _file.read(strPtr->data(), buffSz); _bytesLeft -= buffSz; + LOGS(_log, LOG_LVL_WARN, "&&& _readFromFile() _bytesLeft=" << _bytesLeft); return strPtr; } +CsvStrMemDisk::~CsvStrMemDisk() { + if (_fState != INIT) { + LOGS(_log, LOG_LVL_WARN, "&&& ~CsvStrMemDisk() remove"); + _file.close(); + std::remove(_filePath.c_str()); + } +} + } // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index cd3cb23d1..dc04c7fcd 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -201,19 +201,19 @@ std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& /// important that `_max` is greater than the maximum result size /// (currently 5GB). This should be checked when calling /// MemoryTracker::setup. -/// MEMORYDISK TODO:UJ - Instead new transfers waiting for memory to be +/// MEMORYDISK - Instead new transfers waiting for memory to be /// freed, most of the data will be written to disk when `_max` is /// reached. The current plan is, per UberJob, to write create a /// few CsvBuffers as is done now, and then write everything to /// disk, and have pop read off disk when it runs out of existing /// CsvBuffers. UberJobs with reasonable result sizes should be /// unaffected. -/// TODO:UJ - Test the different options and reorganize code. -class MemoryTracker { +/// @see CsvStrMemDisk +class TransferTracker { public: - using Ptr = std::shared_ptr; + using Ptr = std::shared_ptr; - MemoryTracker() = delete; + TransferTracker() = delete; enum TransferMethod { STREAM, MEMORY, MEMDISK }; static TransferMethod transferMethodFromString(std::string const& str); @@ -233,7 +233,7 @@ class MemoryTracker { ~MemoryRaii() { _globalMt->_decrTotal(memSize); } size_t const memSize; - friend class MemoryTracker; + friend class TransferTracker; private: /// Only to be called by createRaii(), which locks the mutex. @@ -241,7 +241,8 @@ class MemoryTracker { }; friend class MemoryRaii; - static void setup(std::string const& transferMethod, size_t max, std::string const& directory); + static void setup(std::string const& transferMethod, std::size_t max, std::string const& directory, + std::size_t minBytesInMem, std::size_t maxResultTableSizeBytes); static Ptr get() { return _globalMt; } /// Create a MemoryRaii instance to track `fileSize` bytes, and wait for free memory if `wait` is true. @@ -252,13 +253,17 @@ class MemoryTracker { return _total; } - size_t getMax() const { return _max; } - + std::size_t getMax() const { return _max; } std::string getDirectory() const { return _directory; } + std::size_t getMinBytesInMem() const { return _minBytesInMem; } private: - explicit MemoryTracker(TransferMethod transferMethod, size_t max, std::string const& directory) - : _transferMethod(transferMethod), _max(max), _directory(directory) {} + explicit TransferTracker(TransferMethod transferMethod, std::size_t max, std::string const& directory, + std::size_t minBytesInMem) + : _transferMethod(transferMethod), + _max(max), + _directory(directory), + _minBytesInMem(minBytesInMem) {} /// This function only to be called via createRaii. void _incrTotal(size_t sz); @@ -270,10 +275,11 @@ class MemoryTracker { mutable std::mutex _mtx; std::condition_variable _cv; - size_t _total = 0; + std::size_t _total = 0; TransferMethod const _transferMethod; - size_t const _max; + std::size_t const _max; std::string const _directory; + std::size_t const _minBytesInMem; }; class CsvStrMem : public CsvStream { @@ -303,7 +309,7 @@ class CsvStrMem : public CsvStream { std::atomic _bytesRead{0}; size_t const _expectedBytes; - MemoryTracker::MemoryRaii::Ptr _memRaii; + TransferTracker::MemoryRaii::Ptr _memRaii; }; /// Store transfer data in memory until too much memory is being used. @@ -312,6 +318,11 @@ class CsvStrMem : public CsvStream { /// Collecting data from the worker, writing it to disk, reading /// it back, and merging is expected to be linear, run within a /// single thread. +/// The intention is that most reasonable size requests can be handled +/// within memory, which is highly likely to be the fastest method. +/// If a lot of memory (more than TransferTraker::_max) is being used by +/// all current transfers, then transfers greater than _minBytesInMem +/// will be written to disk until memory is free. class CsvStrMemDisk : public CsvStrMem { public: enum FileState { INIT, OPEN_W, CLOSE_W, OPEN_R, CLOSED }; @@ -323,7 +334,7 @@ class CsvStrMemDisk : public CsvStrMem { CsvStrMemDisk() = delete; CsvStrMemDisk(CsvStrMemDisk const&) = delete; CsvStrMemDisk& operator=(CsvStrMemDisk const&) = delete; - ~CsvStrMemDisk() override = default; + ~CsvStrMemDisk() override; void push(char const* data, std::size_t size) override; @@ -338,16 +349,16 @@ class CsvStrMemDisk : public CsvStrMem { private: CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId); - void _writeTofile(char const* data, std::size_t size); + void _writeToTmpfile(char const* data, std::size_t size); /// Read from the file, which should only happen after all writing has finished. - std::shared_ptr _readFromFile(); + std::shared_ptr _readFromTmpFile(); - bool _mustWriteToFile(); + bool _mustWriteToTmpFile(); /// Have at least on record ready to be pushed unsigned int const _minRecordsSize = 1; - size_t const _minBytesInMem = 10'000'000; // &&& config + std::size_t _minBytesInMem; bool _writingToFile = false; std::string const _directory; @@ -360,7 +371,8 @@ class CsvStrMemDisk : public CsvStrMem { std::fstream _file; std::atomic _fileError = false; - size_t _bytesLeft = 0; + std::size_t _bytesWrittenToTmp = 0; + std::size_t _bytesLeft = 0; }; } // namespace lsst::qserv::mysql From 5a944374ce1e5d17d2940e8ca8dcaa1583779f39 Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 26 Jun 2025 16:51:55 -0700 Subject: [PATCH 04/11] Fixed shared pointer loops associated with CsvStream. --- src/cconfig/CzarConfig.h | 8 +++----- src/ccontrol/MergingHandler.cc | 15 ++++++++++----- src/mysql/CsvBuffer.cc | 32 +++++++++++--------------------- src/rproc/InfileMerger.cc | 12 ++++++++---- 4 files changed, 32 insertions(+), 35 deletions(-) diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 38f74b5cf..8a12296d1 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -297,17 +297,15 @@ class CzarConfig { util::ConfigValTInt::create(_configValMap, "resultdb", "maxhttpconnections", notReq, 2000); CVTIntPtr _oldestResultKeptDays = util::ConfigValTInt::create(_configValMap, "resultdb", "oldestResultKeptDays", notReq, 30); - // This must be larger than _maxTableSizeMB + // This must be larger than _maxTableSizeMB when using the "memory" TransferMethod CVTUIntPtr _resultMaxTransferMemMB = - //&&&util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); - util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 0); + util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); CVTStrPtr _resultTransferMethod = util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memdisk"); CVTStrPtr _resultTransferDir = util::ConfigValTStr::create(_configValMap, "resultdb", "transferDir", notReq, "/tmp"); CVTUIntPtr _resultTransferMinMBInMem = - util::ConfigValTUInt::create(_configValMap, "resultdb", "transferMinMBInMem", notReq, 0); - //&&&util::ConfigValTUInt::create(_configValMap, "resultdb", "transferMinMBInMem", notReq, 10); + util::ConfigValTUInt::create(_configValMap, "resultdb", "transferMinMBInMem", notReq, 10); /// Get all the elements in the css section. CVTStrPtr _cssTechnology = diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index eabc8050a..ff9f0439e 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -224,6 +224,7 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber to_string(transferMethod)); } _csvStream = csvStream; + auto weakCsvStream = _csvStream; // This must be after setting _csvStream to avoid cancelFileMerge() // race issues, and it needs to be before the thread starts. @@ -233,21 +234,23 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } string fileReadErrorMsg; - auto csvLambda = [uberJob, csvStream, fileUrl, fileSize, &fileReadErrorMsg]() { + auto csvLambda = [uberJob, weakCsvStream, fileUrl, fileSize, &fileReadErrorMsg]() { size_t bytesRead = 0; fileReadErrorMsg = ::readHttpFileAndMerge( uberJob, fileUrl, fileSize, - [uberJob, csvStream, fileSize, &bytesRead](char const* buf, uint32_t size) { + [uberJob, weakCsvStream, fileSize, &bytesRead](char const* buf, uint32_t size) { + auto csvStrm = weakCsvStream.lock(); + if (csvStrm == nullptr) return; bool last = false; if (buf == nullptr || size == 0) { last = true; } else { - csvStream->push(buf, size); + csvStrm->push(buf, size); bytesRead += size; last = bytesRead >= fileSize; } if (last) { - csvStream->push(nullptr, 0); + csvStrm->push(nullptr, 0); } }, MergingHandler::_getHttpConnPool()); @@ -255,7 +258,9 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // It may be needed to unblock the table merger which may be still attempting to read // from the CSV stream. if (!fileReadErrorMsg.empty()) { - csvStream->push(nullptr, 0); + auto csvStrm = weakCsvStream.lock(); + if (csvStrm == nullptr) return; + csvStrm->push(nullptr, 0); } }; csvStream->setLambda(csvLambda); diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index bcc3ab15d..e78d2d374 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -339,27 +339,29 @@ class CsvStreamBuffer : public CsvBuffer { if (bufLen == 0) { throw LocalInfileError("CsvStreamBuffer::fetch Can't fetch non-positive bytes"); } + auto csvStrm = _csvStream.lock(); + if (csvStrm == nullptr) return 0; if (_str == nullptr) { - _str = _csvStream->pop(); + _str = csvStrm->pop(); _offset = 0; } if (_str->empty()) return 0; if (_offset >= _str->size()) { - _str = _csvStream->pop(); + _str = csvStrm->pop(); _offset = 0; if (_str->empty()) return 0; } unsigned const bytesToCopy = min(bufLen, static_cast(_str->size() - _offset)); ::memcpy(buffer, _str->data() + _offset, bytesToCopy); _offset += bytesToCopy; - _csvStream->increaseBytesWrittenBy(bytesToCopy); + csvStrm->increaseBytesWrittenBy(bytesToCopy); return bytesToCopy; } string dump() const override { return "CsvStreamBuffer"; } private: - shared_ptr _csvStream; + weak_ptr _csvStream; shared_ptr _str; size_t _offset = 0; }; @@ -528,11 +530,11 @@ void CsvStrMemDisk::push(char const* data, size_t size) { // Push is always ok, no need to wait. if (_cancelled) return; _bytesRead += size; + if (_mustWriteToTmpFile()) { + _writeToTmpfile(data, size); + return; + } if (data != nullptr && size != 0) { - if (_mustWriteToTmpFile()) { - _writeToTmpfile(data, size); - return; - } _records.emplace_back(make_shared(data, size)); } else { // Empty string is meant to indicate the end of the stream. @@ -541,7 +543,6 @@ void CsvStrMemDisk::push(char const* data, size_t size) { } shared_ptr CsvStrMemDisk::pop() { - LOGS(_log, LOG_LVL_WARN, "&&& pop " << _records.size()); if (_records.size() > 0) { shared_ptr front = _records.front(); _records.pop_front(); @@ -551,11 +552,9 @@ shared_ptr CsvStrMemDisk::pop() { } void CsvStrMemDisk::_writeToTmpfile(char const* data, std::size_t size) { - LOGS(_log, LOG_LVL_WARN, "&&& _writeToTmpFile() _fState=" << _fState); // Open the file if needed auto oldState = _fState.exchange(OPEN_W); if (oldState == INIT) { - LOGS(_log, LOG_LVL_WARN, "&&& _writeToTmpFile() open out _fState=" << _fState); _file.open(_filePath, fstream::out); } if (!_file.is_open() || _fState != OPEN_W) { @@ -567,20 +566,15 @@ void CsvStrMemDisk::_writeToTmpfile(char const* data, std::size_t size) { _file.write(data, size); _bytesWrittenToTmp += size; - LOGS(_log, LOG_LVL_WARN, - "&&& _writeToTmpFile() _bytesWrittenToTmp=" << _bytesWrittenToTmp << " sz=" << size); } std::shared_ptr CsvStrMemDisk::_readFromTmpFile() { - LOGS(_log, LOG_LVL_WARN, "&&& _readFromTmpFile() _fState=" << _fState); if (_fState == OPEN_W) { - LOGS(_log, LOG_LVL_WARN, "&&& _readFromTmpFile() OPEN_W _fState=" << _fState); _fState = CLOSE_W; _file.close(); } auto oldState = _fState.exchange(OPEN_R); if (oldState == CLOSE_W) { - LOGS(_log, LOG_LVL_WARN, "&&& _readFromTmpFile() CLOSE_W _fState=" << _fState); _file.open(_filePath, fstream::in); _bytesLeft = _bytesWrittenToTmp; } @@ -594,20 +588,16 @@ std::shared_ptr CsvStrMemDisk::_readFromTmpFile() { } std::size_t buffSz = std::min(10'000'000ul, _bytesLeft); - LOGS(_log, LOG_LVL_WARN, - "&&& _readFromTmpFile() _bytesLeft=" << _bytesLeft << " buffSz=" << buffSz - << " _fState=" << _fState); auto strPtr = make_shared(); strPtr->resize(buffSz); _file.read(strPtr->data(), buffSz); _bytesLeft -= buffSz; - LOGS(_log, LOG_LVL_WARN, "&&& _readFromFile() _bytesLeft=" << _bytesLeft); return strPtr; } CsvStrMemDisk::~CsvStrMemDisk() { if (_fState != INIT) { - LOGS(_log, LOG_LVL_WARN, "&&& ~CsvStrMemDisk() remove"); + LOGS(_log, LOG_LVL_INFO, "~CsvStrMemDisk() remove " << _filePath); _file.close(); std::remove(_filePath.c_str()); } diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index c027ebecf..32a3b780b 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -228,10 +228,14 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi return true; } - auto csvStrMemDisk = std::dynamic_pointer_cast(csvStream); - if (csvStrMemDisk != nullptr && csvStrMemDisk->isFileError()) { - csvStrMemDisk->cancel(); - return false; + { + auto csvStrMemDisk = std::dynamic_pointer_cast(csvStream); + if (csvStrMemDisk != nullptr && csvStrMemDisk->isFileError()) { + // The file couldn't be opened for writing, so giving up + // now should keep the result table from getting contaminated. + csvStrMemDisk->cancel(); + return false; + } } auto start = std::chrono::system_clock::now(); From 587c76a6cff81dcc3b108ee4a81de14370e53427 Mon Sep 17 00:00:00 2001 From: John Gates Date: Fri, 27 Jun 2025 09:03:16 -0700 Subject: [PATCH 05/11] Fix for LIMIT problem. --- src/ccontrol/MergingHandler.cc | 19 ++++++--------- src/mysql/CsvBuffer.cc | 3 +-- src/qdisp/Executive.cc | 6 +++-- src/rproc/InfileMerger.cc | 43 ++++++++++++++-------------------- src/rproc/InfileMerger.h | 11 ++++----- 5 files changed, 34 insertions(+), 48 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index ff9f0439e..870138a4f 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -234,23 +234,22 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } string fileReadErrorMsg; - auto csvLambda = [uberJob, weakCsvStream, fileUrl, fileSize, &fileReadErrorMsg]() { + auto csvLambda = [&]() { size_t bytesRead = 0; fileReadErrorMsg = ::readHttpFileAndMerge( uberJob, fileUrl, fileSize, - [uberJob, weakCsvStream, fileSize, &bytesRead](char const* buf, uint32_t size) { - auto csvStrm = weakCsvStream.lock(); - if (csvStrm == nullptr) return; + [&](char const* buf, uint32_t size) { + if (csvStream == nullptr) return; bool last = false; if (buf == nullptr || size == 0) { last = true; } else { - csvStrm->push(buf, size); + csvStream->push(buf, size); bytesRead += size; last = bytesRead >= fileSize; } if (last) { - csvStrm->push(nullptr, 0); + csvStream->push(nullptr, 0); } }, MergingHandler::_getHttpConnPool()); @@ -258,9 +257,8 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // It may be needed to unblock the table merger which may be still attempting to read // from the CSV stream. if (!fileReadErrorMsg.empty()) { - auto csvStrm = weakCsvStream.lock(); - if (csvStrm == nullptr) return; - csvStrm->push(nullptr, 0); + if (csvStream == nullptr) return; + csvStream->push(nullptr, 0); } }; csvStream->setLambda(csvLambda); @@ -322,9 +320,6 @@ qdisp::MergeEndStatus MergingHandler::flushHttp(string const& fileUrl, uint64_t "MergingHandler::" << __func__ << " uberJob=" << uberJob->getIdStr() << " fileUrl=" << fileUrl); qdisp::MergeEndStatus mergeStatus = _mergeHttp(uberJob, fileUrl, fileSize); - if (mergeStatus.success) { - _infileMerger->mergeCompleteFor(uberJob->getUjId()); - } return mergeStatus; } diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index e78d2d374..04f09507a 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -469,7 +469,6 @@ void CsvStrMem::waitReadyToRead() { } void CsvStrMem::push(char const* data, size_t size) { - // Push is always ok, no need to wait. if (_cancelled) return; _bytesRead += size; if (data != nullptr && size != 0) { @@ -587,7 +586,7 @@ std::shared_ptr CsvStrMemDisk::_readFromTmpFile() { return make_shared("$"); } - std::size_t buffSz = std::min(10'000'000ul, _bytesLeft); + std::size_t buffSz = std::min(100'000ul, _bytesLeft); auto strPtr = make_shared(); strPtr->resize(buffSz); _file.read(strPtr->data(), buffSz); diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index da4903d7c..d154e4433 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -759,15 +759,17 @@ void Executive::checkLimitRowComplete() { } void Executive::checkResultFileSize(uint64_t fileSize) { + if (_cancelled || isRowLimitComplete()) return; _totalResultFileSize += fileSize; - if (_cancelled) return; size_t const MB_SIZE_BYTES = 1024 * 1024; uint64_t maxResultTableSizeBytes = cconfig::CzarConfig::instance()->getMaxTableSizeMB() * MB_SIZE_BYTES; LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " sz=" << fileSize << " total=" << _totalResultFileSize << " max=" << maxResultTableSizeBytes); - if (_totalResultFileSize > maxResultTableSizeBytes) { + + if ((fileSize > maxResultTableSizeBytes) || + (!_limitSquashApplies && _totalResultFileSize > maxResultTableSizeBytes)) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " total=" << _totalResultFileSize << " max=" << maxResultTableSizeBytes); // _totalResultFileSize may include non zero values from dead UberJobs, diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 32a3b780b..8ec039562 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -158,37 +158,13 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { _queryIdStrSet = true; } -void InfileMerger::mergeCompleteFor(int jobId) { - std::lock_guard resultSzLock(_mtxResultSizeMtx); - _totalResultSize += _perJobResultSize[jobId]; // TODO:UJ this can probably be simplified -} - bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSize, std::shared_ptr const& csvStream) { - UberJobId const uJobId = uberJob->getUjId(); std::string queryIdJobStr = uberJob->getIdStr(); if (!_queryIdStrSet) { _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); } - // Check if the final result size is too large. It should be safe to do this - // here as the only expected errors at this point are failures in transmission. - // Even if there is a failure in transmission, the retry would be expected - // to put the result size over the limit again. - { - lock_guard resultSzLock(_mtxResultSizeMtx); - _perJobResultSize[uJobId] += fileSize; - size_t tResultSize = _totalResultSize + _perJobResultSize[uJobId]; - if (tResultSize > _maxResultTableSizeBytes) { - string str = queryIdJobStr + " cancelling the query, queryResult table " + _mergeTable + - " is too large at " + to_string(tResultSize) + " bytes, max allowed size is " + - to_string(_maxResultTableSizeBytes) + " bytes"; - LOGS(_log, LOG_LVL_ERROR, str); - _error = util::Error(-1, str, -1); - return false; - } - } - TimeCountTracker::CALLBACKFUNC cbf = [](TIMEPOINT start, TIMEPOINT end, double bytes, bool success) { if (!success) return; @@ -220,6 +196,8 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi return true; } + // Need to block here to make sure the result able needs these rows or not. + lock_guard lgFinal(_finalMergeMtx); // Don't merge if the query got cancelled. auto executive = uberJob->getExecutive(); if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete() || @@ -244,7 +222,21 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi auto end = std::chrono::system_clock::now(); auto mergeDur = std::chrono::duration_cast(end - start); LOGS(_log, LOG_LVL_DEBUG, "mergeDur=" << mergeDur.count()); - if (not ret) { + if (ret) { + lock_guard resultSzLock(_mtxResultSizeMtx); + _totalResultSize += fileSize; + size_t tResultSize = _totalResultSize; + /// Check file size here to see if it has gotten too large, this will probably only trip in LIMIT + /// queries. + if (tResultSize > _maxResultTableSizeBytes) { + string str = queryIdJobStr + " cancelling the query, queryResult table " + _mergeTable + + " is too large at " + to_string(tResultSize) + " bytes, max allowed size is " + + to_string(_maxResultTableSizeBytes) + " bytes"; + LOGS(_log, LOG_LVL_ERROR, str); + _error = util::Error(-1, str, -1); + return false; + } + } else { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::merge mysql applyMysql failure"); } LOGS(_log, LOG_LVL_TRACE, "virtFileT=" << virtFileT.getElapsed() << " mergeDur=" << mergeDur.count()); @@ -292,6 +284,7 @@ size_t InfileMerger::getTotalResultSize() const { return _totalResultSize; } bool InfileMerger::finalize(size_t& collectedBytes, int64_t& rowCount) { bool finalizeOk = true; collectedBytes = _totalResultSize; + lock_guard lgFinal(_finalMergeMtx); // block on other merges // TODO: Should check for error condition before continuing. if (_isFinished) { LOGS(_log, LOG_LVL_ERROR, "InfileMerger::finalize(), but _isFinished == true"); diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index a4e9a5b5d..f3f56f83e 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -109,9 +109,6 @@ class InfileMerger { bool mergeHttp(std::shared_ptr const& uberJob, uint64_t fileSize, std::shared_ptr const& csvStream); - /// Indicate the merge for the job is complete. - void mergeCompleteFor(int jobId); - /// @return error details if finalize() returns false util::Error const& getError() const { return _error; } /// @return final target table name storing results after post processing @@ -189,10 +186,10 @@ class InfileMerger { 10; ///< maximum number of times to retry connecting to the SQL database. /// Variable to track result size. Each - size_t const _maxResultTableSizeBytes; ///< Max result table size in bytes. - size_t _totalResultSize = 0; ///< Size of result so far in bytes. - std::map _perJobResultSize; ///< Result size for each job - std::mutex _mtxResultSizeMtx; ///< Protects _perJobResultSize and _totalResultSize. + size_t const _maxResultTableSizeBytes; ///< Max result table size in bytes. + size_t _totalResultSize = 0; ///< Size of result so far in bytes. + std::mutex _mtxResultSizeMtx; ///< Protects _perJobResultSize and _totalResultSize. + std::mutex _finalMergeMtx; ///< Protects mysql result tables }; } // namespace lsst::qserv::rproc From 2b8e83aa9b0fad8d642048af0f5f7b53cbf5727f Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 1 Jul 2025 11:40:02 -0700 Subject: [PATCH 06/11] Fix for large LIMIT queries. --- src/cconfig/CzarConfig.h | 2 +- src/ccontrol/MergingHandler.cc | 1 - src/qdisp/Executive.h | 7 +++++-- src/qdisp/UberJob.cc | 10 ++++++++-- src/qdisp/UberJob.h | 8 +++++++- 5 files changed, 21 insertions(+), 7 deletions(-) diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 8a12296d1..79a19d2eb 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -301,7 +301,7 @@ class CzarConfig { CVTUIntPtr _resultMaxTransferMemMB = util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); CVTStrPtr _resultTransferMethod = - util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memdisk"); + util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memory"); CVTStrPtr _resultTransferDir = util::ConfigValTStr::create(_configValMap, "resultdb", "transferDir", notReq, "/tmp"); CVTUIntPtr _resultTransferMinMBInMem = diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 870138a4f..57d58dd69 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -224,7 +224,6 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber to_string(transferMethod)); } _csvStream = csvStream; - auto weakCsvStream = _csvStream; // This must be after setting _csvStream to avoid cancelFileMerge() // race issues, and it needs to be before the thread starts. diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 641aea657..872d77e52 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -176,6 +176,8 @@ class Executive : public std::enable_shared_from_this { /// a value <= 0 there's no limit that can be applied at the worker. int getUjRowLimit() const; + bool getLimitSquashApplies() const { return _limitSquashApplies; } + /// @return _rowLimitComplete, which can only be meaningful if the /// user query has not been cancelled. bool isRowLimitComplete() { return _rowLimitComplete && !_cancelled; } @@ -268,8 +270,9 @@ class Executive : public std::enable_shared_from_this { /// How many jobs are used in this query. 1 avoids possible 0 of 0 jobs completed race condition. /// The correct value is set when it is available. std::atomic _totalJobs{1}; - std::shared_ptr - _qdispPool; ///< Shared thread pool for handling commands to and from workers. + + /// Shared thread pool for handling commands to and from workers. + std::shared_ptr _qdispPool; std::deque> _jobStartCmdList; ///< list of jobs to start. diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index 4e1ec6e95..d36314339 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -27,7 +27,6 @@ #include // Third-party headers -#include #include "nlohmann/json.hpp" // Qserv headers @@ -295,16 +294,23 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ } weak_ptr ujThis = weak_from_this(); + bool const limitSquash = exec->getLimitSquashApplies(); // fileCollectFunc will be put on the queue to run later. string const idStr = _idStr; - auto fileCollectFunc = [ujThis, fileUrl, fileSize, rowCount, idStr](util::CmdData*) { + auto fileCollectFunc = [ujThis, fileUrl, fileSize, rowCount, idStr, limitSquash](util::CmdData*) { auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { LOGS(_log, LOG_LVL_DEBUG, "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); return; } + + // Limit collecting LIMIT queries to one at a time, but only those. + shared_ptr> limitSquashL; + if (limitSquash) { + limitSquashL.reset(new lock_guard(ujPtr->_mtxLimitSquash)); + } MergeEndStatus flushStatus = ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize); LOGS(_log, LOG_LVL_TRACE, ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushStatus.success diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index 313a4c740..ccf96f6ef 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -160,11 +160,17 @@ class UberJob : public std::enable_shared_from_this { QueryId const _queryId; UberJobId const _uberJobId; qmeta::CzarId const _czarId; - int const _rowLimit; + int const _rowLimit; ///< Number of rows in the query LIMIT clause. uint64_t _resultFileSize = 0; std::string const _idStr; + /// This mutex is used to limit collecting result files to one at a time + /// but only when the executive will squash the query when the limit is reached. + /// This keeps data transfers (and temporary storage requirements) from + /// getting out of hand. + std::mutex _mtxLimitSquash; + // Map of workerData czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // TODO:UJ this may not be needed From cb40f0c68c103d51ec0b0a04d3eb8e1ae9291b02 Mon Sep 17 00:00:00 2001 From: John Gates Date: Tue, 1 Jul 2025 17:00:55 -0700 Subject: [PATCH 07/11] Removed "stream" and "memory" csv file transfer methods. --- src/cconfig/CzarConfig.h | 4 - src/ccontrol/MergingHandler.cc | 56 ++------ src/ccontrol/MergingHandler.h | 6 +- src/czar/Czar.cc | 5 +- src/mysql/CMakeLists.txt | 1 + src/mysql/CsvBuffer.cc | 232 ------------------------------ src/mysql/CsvBuffer.h | 193 ------------------------- src/mysql/CsvMemDisk.cc | 248 +++++++++++++++++++++++++++++++++ src/mysql/CsvMemDisk.h | 210 ++++++++++++++++++++++++++++ src/qdisp/UberJob.cc | 2 +- src/rproc/InfileMerger.cc | 25 ++-- src/rproc/InfileMerger.h | 4 +- 12 files changed, 489 insertions(+), 497 deletions(-) create mode 100644 src/mysql/CsvMemDisk.cc create mode 100644 src/mysql/CsvMemDisk.h diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 79a19d2eb..785ec566f 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -125,8 +125,6 @@ class CzarConfig { int getMaxTableSizeMB() const { return _maxTableSizeMB->getVal(); } int getMaxSqlConnectionAttempts() const { return _maxSqlConnectionAttempts->getVal(); } unsigned int getMaxTransferMemMB() const { return _resultMaxTransferMemMB->getVal(); } - /// May be "stream", "memory", or "memdisk" - std::string getTransferMethod() const { return _resultTransferMethod->getVal(); } /// Return the transfer directory, which defaults to /tmp, which is bad for performance. std::string getTransferDir() const { return _resultTransferDir->getVal(); } @@ -300,8 +298,6 @@ class CzarConfig { // This must be larger than _maxTableSizeMB when using the "memory" TransferMethod CVTUIntPtr _resultMaxTransferMemMB = util::ConfigValTUInt::create(_configValMap, "resultdb", "maxTransferMemMB", notReq, 10000); - CVTStrPtr _resultTransferMethod = - util::ConfigValTStr::create(_configValMap, "resultdb", "transferMethod", notReq, "memory"); CVTStrPtr _resultTransferDir = util::ConfigValTStr::create(_configValMap, "resultdb", "transferDir", notReq, "/tmp"); CVTUIntPtr _resultTransferMinMBInMem = diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index 57d58dd69..e664c18e4 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -45,7 +45,7 @@ #include "http/Client.h" #include "http/ClientConnPool.h" #include "http/Method.h" -#include "mysql/CsvBuffer.h" +#include "mysql/CsvMemDisk.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qdisp/JobQuery.h" @@ -195,35 +195,8 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } if (fileSize == 0) return qdisp::MergeEndStatus(true); - - // Read from the http stream and push records into the CSV stream in a separate thread. - // Note the fixed capacity of the stream which allows up to 2 records to be buffered - // in the stream. This is enough to hide the latency of the HTTP connection and - // the time needed to read the file. - auto transferMethod = mysql::TransferTracker::get()->getTransferMethod(); - mysql::CsvStream::Ptr csvStream; - switch (transferMethod) { - case mysql::TransferTracker::MEMORY: - // This version should keep the result table valid even if there's - // a worker failure mid data transmission, but may block - // starting transfers when too much memory is in use. - csvStream = mysql::CsvStrMem::create(fileSize); - break; - case mysql::TransferTracker::MEMDISK: - // This version should keep the result table valid even if there's - // a worker failure mid data transmission, but may start writing - // transfer data to disk when memory is low. - csvStream = mysql::CsvStrMemDisk::create(fileSize, uberJob->getQueryId(), uberJob->getUjId()); - break; - case mysql::TransferTracker::STREAM: - // This may contaminate the result table if a worker fails. - csvStream = mysql::CsvStream::create(2); - break; - default: - throw util::Bug(ERR_LOC, "MergingHandler::_mergeHttp UNKNOWN transferMethod " + - to_string(transferMethod)); - } - _csvStream = csvStream; + auto csvMemDisk = mysql::CsvMemDisk::create(fileSize, uberJob->getQueryId(), uberJob->getUjId()); + _csvMemDisk = csvMemDisk; // This must be after setting _csvStream to avoid cancelFileMerge() // race issues, and it needs to be before the thread starts. @@ -233,22 +206,22 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber } string fileReadErrorMsg; - auto csvLambda = [&]() { + auto transferFunc = [&]() { size_t bytesRead = 0; fileReadErrorMsg = ::readHttpFileAndMerge( uberJob, fileUrl, fileSize, [&](char const* buf, uint32_t size) { - if (csvStream == nullptr) return; + if (csvMemDisk == nullptr) return; bool last = false; if (buf == nullptr || size == 0) { last = true; } else { - csvStream->push(buf, size); + csvMemDisk->push(buf, size); bytesRead += size; last = bytesRead >= fileSize; } if (last) { - csvStream->push(nullptr, 0); + csvMemDisk->push(nullptr, 0); } }, MergingHandler::_getHttpConnPool()); @@ -256,26 +229,25 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // It may be needed to unblock the table merger which may be still attempting to read // from the CSV stream. if (!fileReadErrorMsg.empty()) { - if (csvStream == nullptr) return; - csvStream->push(nullptr, 0); + if (csvMemDisk == nullptr) return; + csvMemDisk->push(nullptr, 0); } }; - csvStream->setLambda(csvLambda); + csvMemDisk->transferDataFromWorker(transferFunc); // Attempt the actual merge. - bool fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvStream); + bool fileMergeSuccess = _infileMerger->mergeHttp(uberJob, fileSize, csvMemDisk); if (!fileMergeSuccess) { LOGS(_log, LOG_LVL_WARN, __func__ << " merge failed"); util::Error const& err = _infileMerger->getError(); _setError(ccontrol::MSG_RESULT_ERROR, err.getMsg(), util::ErrorCode::RESULT_IMPORT); } - if (csvStream->getContaminated()) { + if (csvMemDisk->getContaminated()) { LOGS(_log, LOG_LVL_ERROR, __func__ << " merge stream contaminated"); fileMergeSuccess = false; _setError(ccontrol::MSG_RESULT_ERROR, "merge stream contaminated", util::ErrorCode::RESULT_IMPORT); } - csvStream->join(); if (!fileReadErrorMsg.empty()) { LOGS(_log, LOG_LVL_WARN, __func__ << " result file read failed"); _setError(ccontrol::MSG_HTTP_RESULT, fileReadErrorMsg, util::ErrorCode::RESULT_IMPORT); @@ -286,14 +258,14 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber if (!mergeEStatus.success) { // This error check needs to come after the csvThread.join() to ensure writing // is finished. If any bytes were written, the result table is ruined. - mergeEStatus.contaminated = csvStream->getBytesWritten() > 0; + mergeEStatus.contaminated = csvMemDisk->getBytesFetched() > 0; } return mergeEStatus; } void MergingHandler::cancelFileMerge() { - auto csvStrm = _csvStream.lock(); + auto csvStrm = _csvMemDisk.lock(); if (csvStrm != nullptr) { csvStrm->cancel(); } diff --git a/src/ccontrol/MergingHandler.h b/src/ccontrol/MergingHandler.h index a8c0d9012..0f3afc336 100644 --- a/src/ccontrol/MergingHandler.h +++ b/src/ccontrol/MergingHandler.h @@ -38,7 +38,7 @@ class ClientConnPool; } // namespace lsst::qserv::http namespace lsst::qserv::mysql { -class CsvStream; +class CsvMemDisk; } // namespace lsst::qserv::mysql namespace lsst::qserv::qdisp { @@ -105,8 +105,8 @@ class MergingHandler : public qdisp::ResponseHandler { bool _flushed{false}; ///< flushed to InfileMerger? std::string _wName{"~"}; ///< worker name - std::weak_ptr _executive; ///< Weak pointer to the executive for errors. - std::weak_ptr _csvStream; ///< Weak pointer to cancel infile merge. + std::weak_ptr _executive; ///< Weak pointer to the executive for errors. + std::weak_ptr _csvMemDisk; ///< Weak pointer to cancel infile merge. }; } // namespace lsst::qserv::ccontrol diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index 60638c5e7..dad08c305 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -54,7 +54,7 @@ #include "http/ClientConnPool.h" #include "http/MetaModule.h" #include "http/Method.h" -#include "mysql/CsvBuffer.h" +#include "mysql/CsvMemDisk.h" #include "qdisp/CzarStats.h" #include "qdisp/Executive.h" #include "qproc/DatabaseModels.h" @@ -176,10 +176,9 @@ Czar::Czar(string const& configFilePath, string const& czarName) size_t maxResultTableSizeBytes = _czarConfig->getMaxTableSizeMB() * MB_SIZE_BYTES; size_t maxMemToUse = _czarConfig->getMaxTransferMemMB() * MB_SIZE_BYTES; - string const transferMethod = _czarConfig->getTransferMethod(); string const transferDirectory = _czarConfig->getTransferDir(); std::size_t const transferMinBytesInMem = _czarConfig->getTransferMinMBInMem() * MB_SIZE_BYTES; - mysql::TransferTracker::setup(transferMethod, maxMemToUse, transferDirectory, transferMinBytesInMem, + mysql::TransferTracker::setup(maxMemToUse, transferDirectory, transferMinBytesInMem, maxResultTableSizeBytes); auto databaseModels = qproc::DatabaseModels::create(_czarConfig->getCssConfigMap(), diff --git a/src/mysql/CMakeLists.txt b/src/mysql/CMakeLists.txt index 23e418d93..6aa2007a5 100644 --- a/src/mysql/CMakeLists.txt +++ b/src/mysql/CMakeLists.txt @@ -2,6 +2,7 @@ add_library(mysql SHARED) target_sources(mysql PRIVATE CsvBuffer.cc + CsvMemDisk.cc LocalInfile.cc MySqlConfig.cc MySqlConnection.cc diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index 04f09507a..38142f3b0 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -370,236 +370,4 @@ shared_ptr newCsvStreamBuffer(shared_ptr const& csvStream) return make_shared(csvStream); } -TransferTracker::Ptr TransferTracker::_globalMt; - -void TransferTracker::setup(string const& transferMethodStr, std::size_t max, string const& directory, - std::size_t minMBInMem, std::size_t maxResultTableSizeBytes) { - if (_globalMt != nullptr) { - throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); - } - TransferMethod tm = transferMethodFromString(transferMethodStr); - if (tm == MEMDISK) { - if (!verifyDir(directory)) { - throw util::Bug(ERR_LOC, "MemoryTracker::setup called with bad directory! " + directory); - } - } - size_t const MB_SIZE_BYTES = 1024 * 1024; - if (tm == MEMORY && max < maxResultTableSizeBytes) { - throw util::Bug(ERR_LOC, - "Configuration error resultdb maxTransferMemMB=" + to_string(max / MB_SIZE_BYTES) + - " must be larger than maxtablesize_mb= " + - to_string(maxResultTableSizeBytes / MB_SIZE_BYTES)); - } - _globalMt = TransferTracker::Ptr(new TransferTracker(tm, max, directory, minMBInMem)); -} - -bool TransferTracker::verifyDir(string const& dirName) { - sfs::path dir = dirName; - if (!(sfs::exists(dir) && sfs::is_directory(dir))) { - LOGS(_log, LOG_LVL_ERROR, "verifyDir, " + dirName + " is not a valid directory"); - return false; - } - return true; -} - -TransferTracker::TransferMethod TransferTracker::transferMethodFromString(string const& strType) { - string str; - for (unsigned char c : strType) { - str += tolower(c); - } - TransferMethod tMethod; - if (str == "memory") { - tMethod = MEMORY; - LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to memory"); - } else if (str == "stream") { - tMethod = STREAM; - LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to stream"); - } else if (str == "memdisk") { - tMethod = MEMDISK; - LOGS(_log, LOG_LVL_INFO, "Result TransferMethod set to memdisk"); - } else { - tMethod = MEMORY; - LOGS(_log, LOG_LVL_ERROR, - "Result TransferMethod set to memory due to invalid string '" - << strType << "'" - << " valid entries are 'memory', 'stream', 'memdisk'"); - } - return tMethod; -} - -TransferTracker::MemoryRaii::Ptr TransferTracker::createRaii(size_t fileSize, bool wait) { - unique_lock ulck(_mtx); - if (wait) { - if (fileSize > _max) { - throw util::Bug(ERR_LOC, "MemoryTracker::createRaii file too large " + to_string(fileSize) + - " max=" + to_string(_max)); - } - _cv.wait(ulck, [this, fileSize]() { return (fileSize + _total) < _max; }); - } - MemoryRaii::Ptr pRaii(new MemoryRaii(fileSize)); - return pRaii; -} - -void TransferTracker::_incrTotal(size_t sz) { - // _mtx must already be locked. - _total += sz; - _cv.notify_one(); // Many items may be waiting on a large file, so there may be room for more -} - -void TransferTracker::_decrTotal(size_t sz) { - lock_guard ulck(_mtx); - if (sz > _total) { - throw util::Bug(ERR_LOC, - "MemoryTracker::_decrTotal sz=" + to_string(sz) + " > total=" + to_string(_total)); - } - _total -= sz; - _cv.notify_one(); -} - -void CsvStrMem::waitReadyToRead() { - auto memTrack = TransferTracker::get(); - if (memTrack == nullptr) { - throw util::Bug(ERR_LOC, "CsvStrMem::waitReadyToRead MemoryTracker is NULL"); - } - bool const wait = true; - _memRaii = memTrack->createRaii(_expectedBytes, wait); - - // Read directly without starting a separate thread. - _csvLambda(); -} - -void CsvStrMem::push(char const* data, size_t size) { - if (_cancelled) return; - _bytesRead += size; - if (data != nullptr && size != 0) { - _records.emplace_back(make_shared(data, size)); - } else { - // Empty string is meant to indicate the end of the stream. - _records.emplace_back(make_shared()); - } -} - -shared_ptr CsvStrMem::pop() { - shared_ptr front = _records.front(); - _records.pop_front(); - return front; -} - -CsvStrMemDisk::CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId) - : CsvStrMem(expectedBytes), _qId(qId), _ujId(ujId) { - auto memTrack = TransferTracker::get(); - if (memTrack == nullptr) { - throw util::Bug(ERR_LOC, "CsvStrMemDisk constructor MemoryTracker is NULL"); - } - sfs::path fPath = memTrack->getDirectory(); - string fileName = memTrack->getBaseFileName() + "_" + to_string(_qId) + "_" + to_string(ujId); - fPath /= fileName; - _filePath = fPath; - - _minBytesInMem = memTrack->getMinBytesInMem(); -} - -void CsvStrMemDisk::waitReadyToRead() { - auto memTrack = TransferTracker::get(); - if (memTrack == nullptr) { - throw util::Bug(ERR_LOC, "CsvStrMemDisk::waitReadyToRead MemoryTracker is NULL"); - } - bool const nowait = false; - _memRaii = memTrack->createRaii(_expectedBytes, nowait); - - // Read directly without starting a separate thread. - _csvLambda(); -} - -bool CsvStrMemDisk::_mustWriteToTmpFile() { - // Once writing to file, this instance must keep writing to file. - if (_writingToFile) return true; - - auto memTrack = TransferTracker::get(); - // If too much memory is being used for transfers, star writing large transfers to files. - if (memTrack->getTotal() > memTrack->getMax()) { - if (_records.size() > _minRecordsSize && _bytesRead > _minBytesInMem) { - _writingToFile = true; - } - } - return _writingToFile; -} - -void CsvStrMemDisk::push(char const* data, size_t size) { - // Push is always ok, no need to wait. - if (_cancelled) return; - _bytesRead += size; - if (_mustWriteToTmpFile()) { - _writeToTmpfile(data, size); - return; - } - if (data != nullptr && size != 0) { - _records.emplace_back(make_shared(data, size)); - } else { - // Empty string is meant to indicate the end of the stream. - _records.emplace_back(make_shared()); - } -} - -shared_ptr CsvStrMemDisk::pop() { - if (_records.size() > 0) { - shared_ptr front = _records.front(); - _records.pop_front(); - return front; - } - return _readFromTmpFile(); -} - -void CsvStrMemDisk::_writeToTmpfile(char const* data, std::size_t size) { - // Open the file if needed - auto oldState = _fState.exchange(OPEN_W); - if (oldState == INIT) { - _file.open(_filePath, fstream::out); - } - if (!_file.is_open() || _fState != OPEN_W) { - LOGS(_log, LOG_LVL_ERROR, - "CsvStrMemDisk::_writeTofile file isn't open " << _filePath << " or bad state=" << _fState); - _fileError = true; - return; - } - - _file.write(data, size); - _bytesWrittenToTmp += size; -} - -std::shared_ptr CsvStrMemDisk::_readFromTmpFile() { - if (_fState == OPEN_W) { - _fState = CLOSE_W; - _file.close(); - } - auto oldState = _fState.exchange(OPEN_R); - if (oldState == CLOSE_W) { - _file.open(_filePath, fstream::in); - _bytesLeft = _bytesWrittenToTmp; - } - if (!_file.is_open() || _fState != OPEN_R) { - // This is extremely unlikely - if (!getContaminated()) - LOGS(_log, LOG_LVL_ERROR, - "CsvStrMemDisk::_readFromfile file isn't open " << _filePath << " or bad state=" << _fState); - setContaminated(); - return make_shared("$"); - } - - std::size_t buffSz = std::min(100'000ul, _bytesLeft); - auto strPtr = make_shared(); - strPtr->resize(buffSz); - _file.read(strPtr->data(), buffSz); - _bytesLeft -= buffSz; - return strPtr; -} - -CsvStrMemDisk::~CsvStrMemDisk() { - if (_fState != INIT) { - LOGS(_log, LOG_LVL_INFO, "~CsvStrMemDisk() remove " << _filePath); - _file.close(); - std::remove(_filePath.c_str()); - } -} - } // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index dc04c7fcd..39c5f764e 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -109,10 +109,6 @@ class CsvStream { /** * Call to break push operations if the results are no longer needed. * This is only meant to be used to break lingering push() calls. - * TODO:UJ The interleaving of result file reading and table - * merging makes it impossible to guarantee the result - * table is valid in the event that communication - * to a worker is lost during file transfer. * @see UberJob::killUberJob() */ virtual void cancel(); @@ -186,194 +182,5 @@ class CsvStream { */ std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& csvStream); -/// Track how much space is needed to store the current UberJob results while -/// transferring them from the workers and merging them to the result table. -/// How this effects the process depends on the TransferMethod. RAII methods -/// are used to ensure all allocations are freed. -/// STREAM is unaffected by the amount of memory used, but may contaminate -/// the final result table on worker failure, forcing the user query -/// to be cancelled. Not much of an issue with small numbers of workers. -/// MEMORY UberJob results are stored in memory until all data is retrieved -/// from the worker and then merged to the result table. This isolates -/// worker failures from contaminating the result table. The down side -/// is that The result transfer won't be started until the `_total` -/// including the new data is less than the `_max`. It is terribly -/// important that `_max` is greater than the maximum result size -/// (currently 5GB). This should be checked when calling -/// MemoryTracker::setup. -/// MEMORYDISK - Instead new transfers waiting for memory to be -/// freed, most of the data will be written to disk when `_max` is -/// reached. The current plan is, per UberJob, to write create a -/// few CsvBuffers as is done now, and then write everything to -/// disk, and have pop read off disk when it runs out of existing -/// CsvBuffers. UberJobs with reasonable result sizes should be -/// unaffected. -/// @see CsvStrMemDisk -class TransferTracker { -public: - using Ptr = std::shared_ptr; - - TransferTracker() = delete; - - enum TransferMethod { STREAM, MEMORY, MEMDISK }; - static TransferMethod transferMethodFromString(std::string const& str); - static bool verifyDir(std::string const& dirName); - static std::string getBaseFileName() { return std::string("qservtransfer"); } - - /// Return the TransferMethod. - /// @see MemoryTracker - TransferMethod getTransferMethod() const { return _transferMethod; } - - /// This class makes certain that any memory added to MemoryTracker - /// is removed from MemoryTracker. - class MemoryRaii { - public: - using Ptr = std::shared_ptr; - MemoryRaii() = delete; - ~MemoryRaii() { _globalMt->_decrTotal(memSize); } - - size_t const memSize; - friend class TransferTracker; - - private: - /// Only to be called by createRaii(), which locks the mutex. - explicit MemoryRaii(size_t memSize_) : memSize(memSize_) { _globalMt->_incrTotal(memSize); } - }; - friend class MemoryRaii; - - static void setup(std::string const& transferMethod, std::size_t max, std::string const& directory, - std::size_t minBytesInMem, std::size_t maxResultTableSizeBytes); - static Ptr get() { return _globalMt; } - - /// Create a MemoryRaii instance to track `fileSize` bytes, and wait for free memory if `wait` is true. - MemoryRaii::Ptr createRaii(size_t fileSize, bool wait); - - size_t getTotal() const { - std::lock_guard lg(_mtx); - return _total; - } - - std::size_t getMax() const { return _max; } - std::string getDirectory() const { return _directory; } - std::size_t getMinBytesInMem() const { return _minBytesInMem; } - -private: - explicit TransferTracker(TransferMethod transferMethod, std::size_t max, std::string const& directory, - std::size_t minBytesInMem) - : _transferMethod(transferMethod), - _max(max), - _directory(directory), - _minBytesInMem(minBytesInMem) {} - - /// This function only to be called via createRaii. - void _incrTotal(size_t sz); - - /// This function only to be called by ~MemoryRaii() - void _decrTotal(size_t sz); - - static Ptr _globalMt; - - mutable std::mutex _mtx; - std::condition_variable _cv; - std::size_t _total = 0; - TransferMethod const _transferMethod; - std::size_t const _max; - std::string const _directory; - std::size_t const _minBytesInMem; -}; - -class CsvStrMem : public CsvStream { -public: - static std::shared_ptr create(std::size_t expectedBytes) { - return std::shared_ptr(new CsvStrMem(expectedBytes)); - } - - CsvStrMem() = delete; - CsvStrMem(CsvStrMem const&) = delete; - CsvStream& operator=(CsvStream const&) = delete; - ~CsvStrMem() override = default; - - void push(char const* data, std::size_t size) override; - - std::shared_ptr pop() override; - - /// Wait if there isn't enough memory available. - void waitReadyToRead() override; - - /// No thread to join. - void join() override {}; - -protected: - CsvStrMem(std::size_t expectedBytes) : CsvStream(expectedBytes + 1), _expectedBytes(expectedBytes) {}; - - std::atomic _bytesRead{0}; - size_t const _expectedBytes; - - TransferTracker::MemoryRaii::Ptr _memRaii; -}; - -/// Store transfer data in memory until too much memory is being used. -/// By setting the maximum acceptable amount of memory to 0, this -/// effectively becomes writing results to disk. -/// Collecting data from the worker, writing it to disk, reading -/// it back, and merging is expected to be linear, run within a -/// single thread. -/// The intention is that most reasonable size requests can be handled -/// within memory, which is highly likely to be the fastest method. -/// If a lot of memory (more than TransferTraker::_max) is being used by -/// all current transfers, then transfers greater than _minBytesInMem -/// will be written to disk until memory is free. -class CsvStrMemDisk : public CsvStrMem { -public: - enum FileState { INIT, OPEN_W, CLOSE_W, OPEN_R, CLOSED }; - - static std::shared_ptr create(std::size_t expectedBytes, QueryId qId, UberJobId ujId) { - return std::shared_ptr(new CsvStrMemDisk(expectedBytes, qId, ujId)); - } - - CsvStrMemDisk() = delete; - CsvStrMemDisk(CsvStrMemDisk const&) = delete; - CsvStrMemDisk& operator=(CsvStrMemDisk const&) = delete; - ~CsvStrMemDisk() override; - - void push(char const* data, std::size_t size) override; - - std::shared_ptr pop() override; - - /// This version never waits. - void waitReadyToRead() override; - - /// True if a file error happened before results would be contaminated. - bool isFileError() const { return _fileError; } - -private: - CsvStrMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId); - - void _writeToTmpfile(char const* data, std::size_t size); - - /// Read from the file, which should only happen after all writing has finished. - std::shared_ptr _readFromTmpFile(); - - bool _mustWriteToTmpFile(); - - /// Have at least on record ready to be pushed - unsigned int const _minRecordsSize = 1; - std::size_t _minBytesInMem; - - bool _writingToFile = false; - std::string const _directory; - std::string const _baseName; - QueryId const _qId; - UberJobId const _ujId; - - std::atomic _fState = INIT; - std::string _filePath; ///< file path, constant once set. - std::fstream _file; - - std::atomic _fileError = false; - std::size_t _bytesWrittenToTmp = 0; - std::size_t _bytesLeft = 0; -}; - } // namespace lsst::qserv::mysql #endif // LSST_QSERV_MYSQL_CSVBUFFER_H diff --git a/src/mysql/CsvMemDisk.cc b/src/mysql/CsvMemDisk.cc new file mode 100644 index 000000000..9e75763d9 --- /dev/null +++ b/src/mysql/CsvMemDisk.cc @@ -0,0 +1,248 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2014-2015 AURA/LSST. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ + +// Class header +#include "mysql/CsvMemDisk.h" + +// System headers +#include +#include +#include +#include +#include +#include + +// Third-party headers +#include + +// LSST headers +#include "lsst/log/Log.h" + +// Qserv headers +#include "mysql/LocalInfileError.h" +#include "mysql/MySqlUtils.h" +#include "util/Bug.h" + +using namespace std; +namespace sfs = std::filesystem; + +namespace { +LOG_LOGGER _log = LOG_GET("lsst.qserv.mysql.CsvMemDisk"); +} // namespace + +namespace lsst::qserv::mysql { + +TransferTracker::Ptr TransferTracker::_globalMt; + +void TransferTracker::setup(std::size_t max, string const& directory, std::size_t minMBInMem, + std::size_t maxResultTableSizeBytes) { + if (_globalMt != nullptr) { + throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); + } + _globalMt = TransferTracker::Ptr(new TransferTracker(max, directory, minMBInMem)); +} + +bool TransferTracker::verifyDir(string const& dirName) { + sfs::path dir = dirName; + if (!(sfs::exists(dir) && sfs::is_directory(dir))) { + LOGS(_log, LOG_LVL_ERROR, "verifyDir, " + dirName + " is not a valid directory"); + return false; + } + return true; +} + +TransferTracker::MemoryRaii::Ptr TransferTracker::createRaii(size_t fileSize) { + MemoryRaii::Ptr pRaii(new MemoryRaii(fileSize)); + return pRaii; +} + +void TransferTracker::_incrTotal(size_t sz) { + lock_guard ulck(_mtx); + _total += sz; +} + +void TransferTracker::_decrTotal(size_t sz) { + lock_guard ulck(_mtx); + if (sz > _total) { + throw util::Bug(ERR_LOC, + "MemoryTracker::_decrTotal sz=" + to_string(sz) + " > total=" + to_string(_total)); + } + _total -= sz; +} + +CsvMemDisk::CsvMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId) + : _expectedBytes(expectedBytes), _qId(qId), _ujId(ujId) { + auto memTrack = TransferTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMemDisk constructor MemoryTracker is NULL"); + } + sfs::path fPath = memTrack->getDirectory(); + string fileName = memTrack->getBaseFileName() + "_" + to_string(_qId) + "_" + to_string(ujId); + fPath /= fileName; + _filePath = fPath; + _minBytesInMem = memTrack->getMinBytesInMem(); +} + +void CsvMemDisk::transferDataFromWorker(std::function transferFunc) { + auto memTrack = TransferTracker::get(); + if (memTrack == nullptr) { + throw util::Bug(ERR_LOC, "CsvStrMemDisk::waitReadyToRead MemoryTracker is NULL"); + } + _memRaii = memTrack->createRaii(_expectedBytes); + transferFunc(); +} + +bool CsvMemDisk::_mustWriteToTmpFile() { + // Once writing to file, this instance must keep writing to file. + if (_writingToTmpFile) return true; + + auto memTrack = TransferTracker::get(); + // If too much memory is being used for transfers, star writing large transfers to files. + if (memTrack->getTotal() > memTrack->getMax()) { + if (_records.size() > _minRecordsSize && _bytesRead > _minBytesInMem) { + _writingToTmpFile = true; + } + } + return _writingToTmpFile; +} + +void CsvMemDisk::push(char const* data, size_t size) { + // Push is always ok, no need to wait. + if (_cancelled) return; + _bytesRead += size; + if (_mustWriteToTmpFile()) { + _writeToTmpfile(data, size); + return; + } + if (data != nullptr && size != 0) { + _records.emplace_back(make_shared(data, size)); + } else { + // Empty string is meant to indicate the end of the stream. + _records.emplace_back(make_shared()); + } +} + +shared_ptr CsvMemDisk::pop() { + if (_records.size() > 0) { + shared_ptr front = _records.front(); + _records.pop_front(); + return front; + } + return _readFromTmpFile(); +} + +void CsvMemDisk::_writeToTmpfile(char const* data, std::size_t size) { + // Open the file if needed + auto oldState = _fState.exchange(OPEN_W); + if (oldState == INIT) { + _file.open(_filePath, fstream::out); + } + if (!_file.is_open() || _fState != OPEN_W) { + LOGS(_log, LOG_LVL_ERROR, + "CsvStrMemDisk::_writeTofile file isn't open " << _filePath << " or bad state=" << _fState); + _fileError = true; + return; + } + + _file.write(data, size); + _bytesWrittenToTmp += size; +} + +std::shared_ptr CsvMemDisk::_readFromTmpFile() { + if (_fState == OPEN_W) { + _fState = CLOSE_W; + _file.close(); + } + auto oldState = _fState.exchange(OPEN_R); + if (oldState == CLOSE_W) { + _file.open(_filePath, fstream::in); + _bytesLeft = _bytesWrittenToTmp; + } + if (!_file.is_open() || _fState != OPEN_R) { + // This is extremely unlikely and means something has gone wrong with the file system. + // If something has gone wrong with the file system, a crash may be incoming. + if (!getContaminated()) + LOGS(_log, LOG_LVL_ERROR, + "CsvStrMemDisk::_readFromfile file isn't open " << _filePath << " or bad state=" << _fState); + _setContaminated(); + return make_shared("$"); + } + + std::size_t buffSz = std::min(100'000ul, _bytesLeft); + auto strPtr = make_shared(); + strPtr->resize(buffSz); + _file.read(strPtr->data(), buffSz); + _bytesLeft -= buffSz; + return strPtr; +} + +CsvMemDisk::~CsvMemDisk() { + if (_fState != INIT) { + LOGS(_log, LOG_LVL_INFO, "~CsvStrMemDisk() remove " << _filePath); + _file.close(); + std::remove(_filePath.c_str()); + } +} + +class CsvMemDiskBuffer : public CsvBuffer { +public: + explicit CsvMemDiskBuffer(shared_ptr const& csvMemDisk) : _csvMemDisk(csvMemDisk) {} + + ~CsvMemDiskBuffer() override = default; + + unsigned fetch(char* buffer, unsigned bufLen) override { + if (bufLen == 0) { + throw LocalInfileError("CsvStreamBuffer::fetch Can't fetch non-positive bytes"); + } + auto csvStrm = _csvMemDisk.lock(); + if (csvStrm == nullptr) return 0; + if (_str == nullptr) { + _str = csvStrm->pop(); + _offset = 0; + } + if (_str->empty()) return 0; + if (_offset >= _str->size()) { + _str = csvStrm->pop(); + _offset = 0; + if (_str->empty()) return 0; + } + unsigned const bytesToCopy = min(bufLen, static_cast(_str->size() - _offset)); + ::memcpy(buffer, _str->data() + _offset, bytesToCopy); + _offset += bytesToCopy; + csvStrm->increaseBytesFetched(bytesToCopy); + return bytesToCopy; + } + + string dump() const override { return "CsvStreamBuffer"; } + +private: + weak_ptr _csvMemDisk; + shared_ptr _str; + size_t _offset = 0; +}; + +shared_ptr newCsvMemDiskBuffer(shared_ptr const& csvMemDisk) { + return make_shared(csvMemDisk); +} + +} // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvMemDisk.h b/src/mysql/CsvMemDisk.h new file mode 100644 index 000000000..06bdca0ba --- /dev/null +++ b/src/mysql/CsvMemDisk.h @@ -0,0 +1,210 @@ +// -*- LSST-C++ -*- +/* + * LSST Data Management System + * Copyright 2015 LSST Corporation. + * + * This product includes software developed by the + * LSST Project (http://www.lsst.org/). + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the LSST License Statement and + * the GNU General Public License along with this program. If not, + * see . + */ +#ifndef LSST_QSERV_MYSQL_CSVMEMDISK_H +#define LSST_QSERV_MYSQL_CSVMEMDISK_H + +// System headers +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +// Third-party headers +#include + +// qserv headers +#include "mysql/CsvBuffer.h" + +namespace lsst::qserv::mysql { + +/// Track how much space is needed to store the current UberJob results while +/// transferring them from the workers and merging them to the result table. +/// How this effects the process depends on the TransferMethod. RAII methods +/// are used to ensure all allocations are freed. +/// MEMORYDISK - Instead new transfers waiting for memory to be +/// freed, most of the data will be written to disk when `_max` is +/// reached. The current plan is, per UberJob, to write create a +/// few CsvBuffers as is done now, and then write everything to +/// disk, and have pop read off disk when it runs out of existing +/// CsvBuffers. UberJobs with reasonable result sizes should be +/// unaffected. +/// @see CsvStrMemDisk +class TransferTracker { +public: + using Ptr = std::shared_ptr; + + TransferTracker() = delete; + + static bool verifyDir(std::string const& dirName); + static std::string getBaseFileName() { return std::string("qservtransfer"); } + + /// This class makes certain that any memory added to MemoryTracker + /// is removed from MemoryTracker. + class MemoryRaii { + public: + using Ptr = std::shared_ptr; + MemoryRaii() = delete; + ~MemoryRaii() { _globalMt->_decrTotal(memSize); } + + size_t const memSize; + friend class TransferTracker; + + private: + /// Only to be called by createRaii(), which locks the mutex. + explicit MemoryRaii(size_t memSize_) : memSize(memSize_) { _globalMt->_incrTotal(memSize); } + }; + friend class MemoryRaii; + + static void setup(std::size_t max, std::string const& directory, std::size_t minBytesInMem, + std::size_t maxResultTableSizeBytes); + static Ptr get() { return _globalMt; } + + /// Create a MemoryRaii instance to track `fileSize` bytes, and wait for free memory if `wait` is true. + MemoryRaii::Ptr createRaii(size_t fileSize); + + size_t getTotal() const { + std::lock_guard lg(_mtx); + return _total; + } + + std::size_t getMax() const { return _max; } + std::string getDirectory() const { return _directory; } + std::size_t getMinBytesInMem() const { return _minBytesInMem; } + +private: + TransferTracker(std::size_t max, std::string const& directory, std::size_t minBytesInMem) + : _max(max), _directory(directory), _minBytesInMem(minBytesInMem) {} + + /// This function only to be called via createRaii. + void _incrTotal(size_t sz); + + /// This function only to be called by ~MemoryRaii() + void _decrTotal(size_t sz); + + static Ptr _globalMt; + + mutable std::mutex _mtx; + std::size_t _total = 0; + std::size_t const _max; + std::string const _directory; + std::size_t const _minBytesInMem; +}; + +/// Store transfer data in memory until too much memory is being used. +/// By setting the maximum acceptable amount of memory to 0, this +/// effectively becomes writing results to disk. +/// Collecting data from the worker, writing it to disk, reading +/// it back, and merging is expected to be linear, run within a +/// single thread. +/// The intention is that most reasonable size requests can be handled +/// within memory, which is highly likely to be the fastest method. +/// If a lot of memory (more than TransferTraker::_max) is being used by +/// all current transfers, then transfers greater than _minBytesInMem +/// will be written to disk until memory is free. +/// If _contaminated or _fileError get set to true, there are probably +/// catastrophic file system problems. +class CsvMemDisk { +public: + enum FileState { INIT, OPEN_W, CLOSE_W, OPEN_R, CLOSED }; + + static std::shared_ptr create(std::size_t expectedBytes, QueryId qId, UberJobId ujId) { + return std::shared_ptr(new CsvMemDisk(expectedBytes, qId, ujId)); + } + + CsvMemDisk() = delete; + CsvMemDisk(CsvMemDisk const&) = delete; + CsvMemDisk& operator=(CsvMemDisk const&) = delete; + ~CsvMemDisk(); + + void push(char const* data, std::size_t size); + + std::shared_ptr pop(); + + /// This version never waits. + void transferDataFromWorker(std::function transferFunc); + + /// True if a file error happened before results would be contaminated. + bool isFileError() const { return _fileError; } + + /// Stop transferring data before if the query has been cancelled. + void cancel() { _cancelled = true; } + + /// Indicates there was a file system error and the file could not be opened. + bool getContaminated() const { return _contaminated; } + + void increaseBytesFetched(size_t bytesToCopy) { _bytesFetched += bytesToCopy; } + size_t getBytesFetched() const { return _bytesFetched; } + +private: + CsvMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId); + + void _writeToTmpfile(char const* data, std::size_t size); + + /// Read from the file, which should only happen after all writing has finished. + std::shared_ptr _readFromTmpFile(); + + bool _mustWriteToTmpFile(); + + void _setContaminated() { _contaminated = true; } + + std::atomic _cancelled = false; + size_t _bytesFetched = 0; + std::list> _records; + + size_t _bytesRead = 0; + size_t const _expectedBytes; + + /// Indicates there was a file system error and the file could not be opened. + bool _contaminated = false; + + /// Have at least on record ready to be pushed + unsigned int const _minRecordsSize = 1; + std::size_t _minBytesInMem; + + bool _writingToTmpFile = false; + std::string const _directory; + std::string const _baseName; + QueryId const _qId; + UberJobId const _ujId; + + std::atomic _fState = INIT; + std::string _filePath; ///< file path, constant once set. + std::fstream _file; + + bool _fileError = false; + std::size_t _bytesWrittenToTmp = 0; + std::size_t _bytesLeft = 0; + + TransferTracker::MemoryRaii::Ptr _memRaii; +}; + +std::shared_ptr newCsvMemDiskBuffer(std::shared_ptr const& csvMemDisk); + +} // namespace lsst::qserv::mysql +#endif // LSST_QSERV_MYSQL_CSVMEMDISK_H diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index d36314339..d4a33abda 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -306,7 +306,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ return; } - // Limit collecting LIMIT queries to one at a time, but only those. + // Limit collecting LIMIT queries to one(per UJ) at a time, but only those. shared_ptr> limitSquashL; if (limitSquash) { limitSquashL.reset(new lock_guard(ujPtr->_mtxLimitSquash)); diff --git a/src/rproc/InfileMerger.cc b/src/rproc/InfileMerger.cc index 8ec039562..cc5bcd8df 100644 --- a/src/rproc/InfileMerger.cc +++ b/src/rproc/InfileMerger.cc @@ -56,7 +56,7 @@ // Qserv headers #include "cconfig/CzarConfig.h" #include "global/intTypes.h" -#include "mysql/CsvBuffer.h" +#include "mysql/CsvMemDisk.h" #include "qdisp/CzarStats.h" #include "qdisp/UberJob.h" #include "qproc/DatabaseModels.h" @@ -159,7 +159,7 @@ void InfileMerger::_setQueryIdStr(std::string const& qIdStr) { } bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSize, - std::shared_ptr const& csvStream) { + std::shared_ptr const& csvMemDisk) { std::string queryIdJobStr = uberJob->getIdStr(); if (!_queryIdStrSet) { _setQueryIdStr(QueryIdHelper::makeIdStr(uberJob->getQueryId())); @@ -174,14 +174,11 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi }; auto tct = make_shared>(cbf); - // Start downloading. - csvStream->waitReadyToRead(); - bool ret = false; // Add columns to rows in virtFile. util::Timer virtFileT; virtFileT.start(); - auto const csvBuffer = mysql::newCsvStreamBuffer(csvStream); + auto const csvBuffer = mysql::newCsvMemDiskBuffer(csvMemDisk); std::string const virtFile = _infileMgr.prepareSrc(csvBuffer); std::string const infileStatement = sql::formLoadInfile(_mergeTable, virtFile); virtFileT.stop(); @@ -200,20 +197,14 @@ bool InfileMerger::mergeHttp(qdisp::UberJob::Ptr const& uberJob, uint64_t fileSi lock_guard lgFinal(_finalMergeMtx); // Don't merge if the query got cancelled. auto executive = uberJob->getExecutive(); - if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete() || - csvStream->isCancelled()) { - csvStream->cancel(); + if (executive == nullptr || executive->getCancelled() || executive->isRowLimitComplete()) { return true; } - { - auto csvStrMemDisk = std::dynamic_pointer_cast(csvStream); - if (csvStrMemDisk != nullptr && csvStrMemDisk->isFileError()) { - // The file couldn't be opened for writing, so giving up - // now should keep the result table from getting contaminated. - csvStrMemDisk->cancel(); - return false; - } + if (csvMemDisk->isFileError()) { + // The file couldn't be opened for writing, so giving up + // now should keep the result table from getting contaminated. + return false; } auto start = std::chrono::system_clock::now(); diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index f3f56f83e..3e1890459 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -49,7 +49,7 @@ // Forward declarations namespace lsst::qserv { namespace mysql { -class CsvStream; +class CsvMemDisk; class MysqlConfig; } // namespace mysql namespace qdisp { @@ -107,7 +107,7 @@ class InfileMerger { /// Merge the result data collected over Http. bool mergeHttp(std::shared_ptr const& uberJob, uint64_t fileSize, - std::shared_ptr const& csvStream); + std::shared_ptr const& csvMemDisk); /// @return error details if finalize() returns false util::Error const& getError() const { return _error; } From b3a1a18242d746c9b8648842e2b450aa66440192 Mon Sep 17 00:00:00 2001 From: John Gates Date: Wed, 2 Jul 2025 07:43:29 -0700 Subject: [PATCH 08/11] Reverted CsvBuffer. --- src/ccontrol/MergingHandler.cc | 2 - src/mysql/CsvBuffer.cc | 83 ++++++++++++++-------------------- src/mysql/CsvBuffer.h | 53 +++++----------------- src/mysql/CsvMemDisk.cc | 12 ++--- src/mysql/CsvMemDisk.h | 1 + 5 files changed, 53 insertions(+), 98 deletions(-) diff --git a/src/ccontrol/MergingHandler.cc b/src/ccontrol/MergingHandler.cc index e664c18e4..c32612c72 100644 --- a/src/ccontrol/MergingHandler.cc +++ b/src/ccontrol/MergingHandler.cc @@ -211,7 +211,6 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber fileReadErrorMsg = ::readHttpFileAndMerge( uberJob, fileUrl, fileSize, [&](char const* buf, uint32_t size) { - if (csvMemDisk == nullptr) return; bool last = false; if (buf == nullptr || size == 0) { last = true; @@ -229,7 +228,6 @@ qdisp::MergeEndStatus MergingHandler::_mergeHttp(qdisp::UberJob::Ptr const& uber // It may be needed to unblock the table merger which may be still attempting to read // from the CSV stream. if (!fileReadErrorMsg.empty()) { - if (csvMemDisk == nullptr) return; csvMemDisk->push(nullptr, 0); } }; diff --git a/src/mysql/CsvBuffer.cc b/src/mysql/CsvBuffer.cc index 38142f3b0..7113b63ff 100644 --- a/src/mysql/CsvBuffer.cc +++ b/src/mysql/CsvBuffer.cc @@ -27,8 +27,6 @@ // System headers #include #include -#include -#include #include #include @@ -41,16 +39,10 @@ // Qserv headers #include "mysql/LocalInfileError.h" #include "mysql/MySqlUtils.h" -#include "util/Bug.h" - -using namespace std; -namespace sfs = std::filesystem; namespace { -LOG_LOGGER _log = LOG_GET("lsst.qserv.mysql.CsvBuffer"); - -string const mysqlNull("\\N"); +std::string const mysqlNull("\\N"); int const largeRowThreshold = 500 * 1024; // should be less than 0.5 * infileBufferSize } // namespace @@ -84,13 +76,13 @@ inline unsigned updateEstRowSize(unsigned lastRowSize, Row const& r) { return lastRowSize < rowSize ? rowSize : lastRowSize; } -inline int addString(char* cursor, string const& s) { +inline int addString(char* cursor, std::string const& s) { int const sSize = s.size(); memcpy(cursor, s.data(), sSize); return sSize; } -inline int maxColFootprint(int columnLength, string const& sep) { +inline int maxColFootprint(int columnLength, std::string const& sep) { const int overhead = 2 + sep.size(); // NULL decl + sep size return overhead + (2 * columnLength); } @@ -116,7 +108,7 @@ class ResCsvBuffer : public CsvBuffer { bool _fetchRow(Row& r); unsigned _fetchFromLargeRow(char* buffer, int bufLen); void _initializeLargeRow(Row const& largeRow); - string dump() const override; + std::string dump() const override; private: MYSQL_RES* _result; @@ -127,8 +119,8 @@ class ResCsvBuffer : public CsvBuffer { Row _largeRow; int _fieldOffset; - string _sep; - string _rowSep; + std::string _sep; + std::string _rowSep; }; ResCsvBuffer::ResCsvBuffer(MYSQL_RES* result) @@ -139,8 +131,8 @@ ResCsvBuffer::ResCsvBuffer(MYSQL_RES* result) // cout << _numFields << " fields per row\n"; } -string ResCsvBuffer::dump() const { - string str = string("ResCsvBuffer _numFields=") + to_string(_numFields); +std::string ResCsvBuffer::dump() const { + std::string str = std::string("ResCsvBuffer _numFields=") + std::to_string(_numFields); return str; } @@ -258,36 +250,38 @@ void ResCsvBuffer::_initializeLargeRow(Row const& largeRow) { _fieldOffset = 0; } -shared_ptr newResCsvBuffer(MYSQL_RES* result) { return make_shared(result); } +std::shared_ptr newResCsvBuffer(MYSQL_RES* result) { + return std::make_shared(result); +} -CsvStream::CsvStream(size_t maxRecords) : _maxRecords(maxRecords) { +CsvStream::CsvStream(std::size_t maxRecords) : _maxRecords(maxRecords) { if (maxRecords == 0) { - throw invalid_argument("CsvStream::CsvStream: maxRecords must be greater than 0"); + throw std::invalid_argument("CsvStream::CsvStream: maxRecords must be greater than 0"); } } void CsvStream::cancel() { - unique_lock lock(_mtx); + std::unique_lock lock(_mtx); _cancelled = true; _cv.notify_all(); } -void CsvStream::push(char const* data, size_t size) { - unique_lock lock(_mtx); +void CsvStream::push(char const* data, std::size_t size) { + std::unique_lock lock(_mtx); _cv.wait(lock, [this]() { return (_records.size() < _maxRecords) || _cancelled; }); if (_cancelled) return; if (data != nullptr && size != 0) { - _records.emplace_back(make_shared(data, size)); + _records.emplace_back(std::make_shared(data, size)); } else { // Empty string is meant to indicate the end of the stream. - _records.emplace_back(make_shared()); + _records.emplace_back(std::make_shared()); } _cv.notify_one(); } -shared_ptr CsvStream::pop() { - unique_lock lock(_mtx); +std::shared_ptr CsvStream::pop() { + std::unique_lock lock(_mtx); _cv.wait(lock, [this]() { return (!_records.empty() || _cancelled); }); if (_records.empty()) { @@ -298,28 +292,21 @@ shared_ptr CsvStream::pop() { // database stops asking for them. // See CsvStream::cancel() _contaminated = true; - auto pstr = make_shared("$"); + auto pstr = std::make_shared("$"); _cv.notify_one(); return pstr; } - shared_ptr front = _records.front(); + std::shared_ptr front = _records.front(); _records.pop_front(); _cv.notify_one(); return front; } bool CsvStream::empty() const { - unique_lock lock(_mtx); + std::unique_lock lock(_mtx); return _records.empty(); } -void CsvStream::waitReadyToRead() { - // No need to wait for this class - thread thrd(_csvLambda); - _thrd = move(thrd); - _thrdStarted = true; -} - /** * CsvStreamBuffer is a CsvBuffer that reads from a CsvStream. It is used to read * data from a CsvStream in a buffered manner. @@ -331,7 +318,7 @@ void CsvStream::waitReadyToRead() { */ class CsvStreamBuffer : public CsvBuffer { public: - explicit CsvStreamBuffer(shared_ptr const& csvStream) : _csvStream(csvStream) {} + explicit CsvStreamBuffer(std::shared_ptr const& csvStream) : _csvStream(csvStream) {} ~CsvStreamBuffer() override = default; @@ -339,35 +326,33 @@ class CsvStreamBuffer : public CsvBuffer { if (bufLen == 0) { throw LocalInfileError("CsvStreamBuffer::fetch Can't fetch non-positive bytes"); } - auto csvStrm = _csvStream.lock(); - if (csvStrm == nullptr) return 0; if (_str == nullptr) { - _str = csvStrm->pop(); + _str = _csvStream->pop(); _offset = 0; } if (_str->empty()) return 0; if (_offset >= _str->size()) { - _str = csvStrm->pop(); + _str = _csvStream->pop(); _offset = 0; if (_str->empty()) return 0; } - unsigned const bytesToCopy = min(bufLen, static_cast(_str->size() - _offset)); + unsigned const bytesToCopy = std::min(bufLen, static_cast(_str->size() - _offset)); ::memcpy(buffer, _str->data() + _offset, bytesToCopy); _offset += bytesToCopy; - csvStrm->increaseBytesWrittenBy(bytesToCopy); + _csvStream->increaseBytesWrittenBy(bytesToCopy); return bytesToCopy; } - string dump() const override { return "CsvStreamBuffer"; } + std::string dump() const override { return "CsvStreamBuffer"; } private: - weak_ptr _csvStream; - shared_ptr _str; - size_t _offset = 0; + std::shared_ptr _csvStream; + std::shared_ptr _str; + std::size_t _offset = 0; }; -shared_ptr newCsvStreamBuffer(shared_ptr const& csvStream) { - return make_shared(csvStream); +std::shared_ptr newCsvStreamBuffer(std::shared_ptr const& csvStream) { + return std::make_shared(csvStream); } } // namespace lsst::qserv::mysql diff --git a/src/mysql/CsvBuffer.h b/src/mysql/CsvBuffer.h index 39c5f764e..e9abcd113 100644 --- a/src/mysql/CsvBuffer.h +++ b/src/mysql/CsvBuffer.h @@ -24,23 +24,16 @@ #define LSST_QSERV_MYSQL_CSVBUFFER_H // System headers -#include #include #include -#include -#include #include #include #include #include -#include // Third-party headers #include -// qserv headers -#include "global/intTypes.h" - namespace lsst::qserv::mysql { /** @@ -78,8 +71,6 @@ std::shared_ptr newResCsvBuffer(MYSQL_RES* result); */ class CsvStream { public: - using Ptr = std::shared_ptr; - /** * Factory function to create a new CsvStream object. * @param maxRecords The maximum number of records in the stream @@ -93,7 +84,7 @@ class CsvStream { CsvStream() = delete; CsvStream(CsvStream const&) = delete; CsvStream& operator=(CsvStream const&) = delete; - virtual ~CsvStream() = default; + ~CsvStream() = default; /** * Push a new record to the stream. The record is a string of bytes. @@ -104,26 +95,25 @@ class CsvStream { * @param data The record to be pushed to the stream * @param size The size of the record */ - virtual void push(char const* data, std::size_t size); + void push(char const* data, std::size_t size); /** * Call to break push operations if the results are no longer needed. * This is only meant to be used to break lingering push() calls. + * TODO:UJ The interleaving of result file reading and table + * merging makes it impossible to guarantee the result + * table is valid in the event that communication + * to a worker is lost during file transfer. * @see UberJob::killUberJob() */ - virtual void cancel(); - - /* - * Return true if this instance has been cancelled. - */ - bool isCancelled() const { return _cancelled; } + void cancel(); /** * Pop a record from the stream. The method will block if the stream is empty * until a record is pushed. * @return A shared pointer to the popped record or an empty string for the end of the stream */ - virtual std::shared_ptr pop(); + std::shared_ptr pop(); /** * Check if the stream is empty. @@ -143,35 +133,16 @@ class CsvStream { */ bool getContaminated() const { return _contaminated; } - /// The function to run to read/push the data from the worker. - void setLambda(std::function csvLambda) { _csvLambda = csvLambda; } - - /// In this class, no waiting, just start the read/push thread. - virtual void waitReadyToRead(); - - /// Join the thread, must be called from the same thread that called waitReadyToRead - virtual void join() { - if (_thrdStarted) _thrd.join(); - } - -protected: +private: CsvStream(std::size_t maxRecords); - void setContaminated() { _contaminated = true; } - - std::function _csvLambda; - bool _cancelled = false; - std::atomic _bytesWritten; - std::list> _records; - -private: mutable std::mutex _mtx; std::condition_variable _cv; std::size_t const _maxRecords; - + std::list> _records; + std::atomic _bytesWritten; + bool _cancelled = false; std::atomic _contaminated = false; - std::thread _thrd; - bool _thrdStarted = false; }; /** diff --git a/src/mysql/CsvMemDisk.cc b/src/mysql/CsvMemDisk.cc index 9e75763d9..0b7666bbd 100644 --- a/src/mysql/CsvMemDisk.cc +++ b/src/mysql/CsvMemDisk.cc @@ -188,7 +188,7 @@ std::shared_ptr CsvMemDisk::_readFromTmpFile() { return make_shared("$"); } - std::size_t buffSz = std::min(100'000ul, _bytesLeft); + std::size_t buffSz = std::min(1'000'000ul, _bytesLeft); auto strPtr = make_shared(); strPtr->resize(buffSz); _file.read(strPtr->data(), buffSz); @@ -214,22 +214,22 @@ class CsvMemDiskBuffer : public CsvBuffer { if (bufLen == 0) { throw LocalInfileError("CsvStreamBuffer::fetch Can't fetch non-positive bytes"); } - auto csvStrm = _csvMemDisk.lock(); - if (csvStrm == nullptr) return 0; + auto csvMd = _csvMemDisk.lock(); + if (csvMd == nullptr) return 0; if (_str == nullptr) { - _str = csvStrm->pop(); + _str = csvMd->pop(); _offset = 0; } if (_str->empty()) return 0; if (_offset >= _str->size()) { - _str = csvStrm->pop(); + _str = csvMd->pop(); _offset = 0; if (_str->empty()) return 0; } unsigned const bytesToCopy = min(bufLen, static_cast(_str->size() - _offset)); ::memcpy(buffer, _str->data() + _offset, bytesToCopy); _offset += bytesToCopy; - csvStrm->increaseBytesFetched(bytesToCopy); + csvMd->increaseBytesFetched(bytesToCopy); return bytesToCopy; } diff --git a/src/mysql/CsvMemDisk.h b/src/mysql/CsvMemDisk.h index 06bdca0ba..aef9b7f0b 100644 --- a/src/mysql/CsvMemDisk.h +++ b/src/mysql/CsvMemDisk.h @@ -39,6 +39,7 @@ #include // qserv headers +#include "global/intTypes.h" #include "mysql/CsvBuffer.h" namespace lsst::qserv::mysql { From 6465dcbd2837126879ace895d6f85efe7f97825e Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 3 Jul 2025 10:10:05 -0700 Subject: [PATCH 09/11] Added czar id to temporary file names. --- admin/local/docker/compose/docker-compose.yml | 8 ++++++++ src/czar/Czar.cc | 18 +++++++++--------- src/mysql/CsvMemDisk.cc | 7 ++++--- src/mysql/CsvMemDisk.h | 9 ++++++--- 4 files changed, 27 insertions(+), 15 deletions(-) diff --git a/admin/local/docker/compose/docker-compose.yml b/admin/local/docker/compose/docker-compose.yml index 813329969..ebaa89ab4 100644 --- a/admin/local/docker/compose/docker-compose.yml +++ b/admin/local/docker/compose/docker-compose.yml @@ -36,6 +36,7 @@ volumes: volume_czar_xrootd: volume_czar_home: volume_czar_cfg: + volume_czar_tmp: volume_czar_mariadb_data: volume_czar_mariadb_cfg: @@ -272,6 +273,10 @@ services: - type: volume source: volume_czar_mariadb_run target: /qserv/mariadb/run + - type: volume + source: volume_czar_tmp + target: /tmp + - << : *log-volume expose: - "3306" # for czar-mariadb @@ -306,6 +311,9 @@ services: - type: volume source: volume_czar_cfg target: /config-etc + - type: volume + source: volume_czar_tmp + target: /tmp - type: volume source: volume_czar_home target: /home/qserv diff --git a/src/czar/Czar.cc b/src/czar/Czar.cc index dad08c305..fa7e84d7b 100644 --- a/src/czar/Czar.cc +++ b/src/czar/Czar.cc @@ -172,15 +172,6 @@ Czar::Czar(string const& configFilePath, string const& czarName) const int year = 60 * 60 * 24 * 365; _idCounter = uint64_t(tv.tv_sec % year) * 1000 + tv.tv_usec / 1000; - size_t const MB_SIZE_BYTES = 1024 * 1024; - size_t maxResultTableSizeBytes = _czarConfig->getMaxTableSizeMB() * MB_SIZE_BYTES; - - size_t maxMemToUse = _czarConfig->getMaxTransferMemMB() * MB_SIZE_BYTES; - string const transferDirectory = _czarConfig->getTransferDir(); - std::size_t const transferMinBytesInMem = _czarConfig->getTransferMinMBInMem() * MB_SIZE_BYTES; - mysql::TransferTracker::setup(maxMemToUse, transferDirectory, transferMinBytesInMem, - maxResultTableSizeBytes); - auto databaseModels = qproc::DatabaseModels::create(_czarConfig->getCssConfigMap(), _czarConfig->getMySqlResultConfig()); @@ -191,6 +182,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); + CzarIdType czarId = _czarConfig->id(); + size_t const MB_SIZE_BYTES = 1024 * 1024; + size_t maxResultTableSizeBytes = _czarConfig->getMaxTableSizeMB() * MB_SIZE_BYTES; + size_t maxMemToUse = _czarConfig->getMaxTransferMemMB() * MB_SIZE_BYTES; + string const transferDirectory = _czarConfig->getTransferDir(); + std::size_t const transferMinBytesInMem = _czarConfig->getTransferMinMBInMem() * MB_SIZE_BYTES; + mysql::TransferTracker::setup(maxMemToUse, transferDirectory, transferMinBytesInMem, + maxResultTableSizeBytes, czarId); + // Tell workers to cancel any queries that were submitted before this restart of Czar. // Figure out which query (if any) was recorded in Czar databases before the restart. // The id will be used as the high-watermark for queries that need to be cancelled. diff --git a/src/mysql/CsvMemDisk.cc b/src/mysql/CsvMemDisk.cc index 0b7666bbd..d2c7eafef 100644 --- a/src/mysql/CsvMemDisk.cc +++ b/src/mysql/CsvMemDisk.cc @@ -55,11 +55,11 @@ namespace lsst::qserv::mysql { TransferTracker::Ptr TransferTracker::_globalMt; void TransferTracker::setup(std::size_t max, string const& directory, std::size_t minMBInMem, - std::size_t maxResultTableSizeBytes) { + std::size_t maxResultTableSizeBytes, CzarIdType czarId) { if (_globalMt != nullptr) { throw util::Bug(ERR_LOC, "MemoryTracker::setup called when MemoryTracker already setup!"); } - _globalMt = TransferTracker::Ptr(new TransferTracker(max, directory, minMBInMem)); + _globalMt = TransferTracker::Ptr(new TransferTracker(max, directory, minMBInMem, czarId)); } bool TransferTracker::verifyDir(string const& dirName) { @@ -97,7 +97,8 @@ CsvMemDisk::CsvMemDisk(std::size_t expectedBytes, QueryId qId, UberJobId ujId) throw util::Bug(ERR_LOC, "CsvStrMemDisk constructor MemoryTracker is NULL"); } sfs::path fPath = memTrack->getDirectory(); - string fileName = memTrack->getBaseFileName() + "_" + to_string(_qId) + "_" + to_string(ujId); + string fileName = memTrack->getBaseFileName() + "_" + to_string(memTrack->getCzarId()) + "_" + + to_string(_qId) + "_" + to_string(ujId); fPath /= fileName; _filePath = fPath; _minBytesInMem = memTrack->getMinBytesInMem(); diff --git a/src/mysql/CsvMemDisk.h b/src/mysql/CsvMemDisk.h index aef9b7f0b..d59f0e34d 100644 --- a/src/mysql/CsvMemDisk.h +++ b/src/mysql/CsvMemDisk.h @@ -83,7 +83,7 @@ class TransferTracker { friend class MemoryRaii; static void setup(std::size_t max, std::string const& directory, std::size_t minBytesInMem, - std::size_t maxResultTableSizeBytes); + std::size_t maxResultTableSizeBytes, CzarIdType czarId); static Ptr get() { return _globalMt; } /// Create a MemoryRaii instance to track `fileSize` bytes, and wait for free memory if `wait` is true. @@ -97,10 +97,12 @@ class TransferTracker { std::size_t getMax() const { return _max; } std::string getDirectory() const { return _directory; } std::size_t getMinBytesInMem() const { return _minBytesInMem; } + CzarIdType getCzarId() const { return _czarId; } private: - TransferTracker(std::size_t max, std::string const& directory, std::size_t minBytesInMem) - : _max(max), _directory(directory), _minBytesInMem(minBytesInMem) {} + TransferTracker(std::size_t max, std::string const& directory, std::size_t minBytesInMem, + CzarIdType czarId) + : _max(max), _directory(directory), _minBytesInMem(minBytesInMem), _czarId(czarId) {} /// This function only to be called via createRaii. void _incrTotal(size_t sz); @@ -115,6 +117,7 @@ class TransferTracker { std::size_t const _max; std::string const _directory; std::size_t const _minBytesInMem; + CzarIdType const _czarId; }; /// Store transfer data in memory until too much memory is being used. From 422696a3485061b54f113ec016efffed6cdeb1bb Mon Sep 17 00:00:00 2001 From: John Gates Date: Thu, 3 Jul 2025 13:00:30 -0700 Subject: [PATCH 10/11] Improved LIMIT fix. --- src/qdisp/Executive.cc | 49 +++++++++++++++++++++++++++++ src/qdisp/Executive.h | 14 +++++++-- src/qdisp/UberJob.cc | 70 ++++++++++-------------------------------- src/qdisp/UberJob.h | 23 +++++--------- 4 files changed, 86 insertions(+), 70 deletions(-) diff --git a/src/qdisp/Executive.cc b/src/qdisp/Executive.cc index d154e4433..b1eb69f10 100644 --- a/src/qdisp/Executive.cc +++ b/src/qdisp/Executive.cc @@ -794,6 +794,55 @@ void Executive::checkResultFileSize(uint64_t fileSize) { } } +shared_ptr> Executive::getLimitSquashLock() { + shared_ptr> ptr(new lock_guard(_mtxLimitSquash)); + return ptr; +} + +void Executive::collectFile(std::shared_ptr ujPtr, std::string const& fileUrl, uint64_t fileSize, + uint64_t rowCount, std::string const& idStr) { + // Limit collecting LIMIT queries to one at a time, but only those. + shared_ptr> limitSquashL; + if (_limitSquashApplies) { + limitSquashL.reset(new lock_guard(_mtxLimitSquash)); + } + MergeEndStatus flushStatus = ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize); + LOGS(_log, LOG_LVL_TRACE, + cName(__func__) << "ujId=" << ujPtr->getUjId() << " success=" << flushStatus.success + << " contaminated=" << flushStatus.contaminated); + if (flushStatus.success) { + CzarStats::get()->addTotalRowsRecv(rowCount); + CzarStats::get()->addTotalBytesRecv(fileSize); + } else { + if (flushStatus.contaminated) { + // This would probably indicate malformed file+rowCount or writing the result table failed. + // If any merging happened, the result table is ruined. + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << "ujId=" << ujPtr->getUjId() + << " flushHttp failed after merging, results ruined."); + } else { + // Perhaps something went wrong with file collection, so it is worth trying the jobs again + // by abandoning this UberJob. + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << "ujId=" << ujPtr->getUjId() << " flushHttp failed, retrying Jobs."); + } + ujPtr->importResultError(flushStatus.contaminated, "mergeError", "merging failed"); + } + + // At this point all data for this job have been read and merged + bool const statusSet = ujPtr->importResultFinish(); + if (!statusSet) { + LOGS(_log, LOG_LVL_ERROR, + cName(__func__) << "ujId=" << ujPtr->getUjId() << " failed to set status, squashing " + << getIdStr()); + // Something has gone very wrong + squash(cName(__func__) + " couldn't set UberJob status"); + return; + } + addResultRows(rowCount); + checkLimitRowComplete(); +} + ostream& operator<<(ostream& os, Executive::JobMap::value_type const& v) { auto const& status = v.second->getStatus(); os << v.first << ": " << *status; diff --git a/src/qdisp/Executive.h b/src/qdisp/Executive.h index 872d77e52..f66f607d4 100644 --- a/src/qdisp/Executive.h +++ b/src/qdisp/Executive.h @@ -176,8 +176,6 @@ class Executive : public std::enable_shared_from_this { /// a value <= 0 there's no limit that can be applied at the worker. int getUjRowLimit() const; - bool getLimitSquashApplies() const { return _limitSquashApplies; } - /// @return _rowLimitComplete, which can only be meaningful if the /// user query has not been cancelled. bool isRowLimitComplete() { return _rowLimitComplete && !_cancelled; } @@ -231,6 +229,12 @@ class Executive : public std::enable_shared_from_this { /// cancel this user query. void checkResultFileSize(uint64_t fileSize = 0); + /// Returns a pointer to a lock on _mtxLimitSquash. + std::shared_ptr> getLimitSquashLock(); + + void collectFile(std::shared_ptr ujPtr, std::string const& fileUrl, uint64_t fileSize, + uint64_t rowCount, std::string const& idStr); + protected: Executive(int secondsBetweenUpdates, std::shared_ptr const& ms, std::shared_ptr const& sharedResources, @@ -347,6 +351,12 @@ class Executive : public std::enable_shared_from_this { std::atomic _totalResultFileSize{0}; ///< Total size of all UberJob result files. std::atomic _jobCancelCount{0}; ///< Total number of JOB_CANCEL messages received. + + /// This mutex is used to limit collecting result files to one at a time + /// but only when the executive will squash the query when the limit is reached. + /// This keeps data transfers (and temporary storage requirements) from + /// getting out of hand. + std::mutex _mtxLimitSquash; }; } // namespace qdisp diff --git a/src/qdisp/UberJob.cc b/src/qdisp/UberJob.cc index d4a33abda..302c9e985 100644 --- a/src/qdisp/UberJob.cc +++ b/src/qdisp/UberJob.cc @@ -268,12 +268,12 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ auto exec = _executive.lock(); if (exec == nullptr) { LOGS(_log, LOG_LVL_WARN, cName(__func__) + " no executive"); - return _importResultError(true, "cancelled", "Query cancelled - no executive"); + return importResultError(true, "cancelled", "Query cancelled - no executive"); } if (exec->getCancelled()) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " import job was cancelled."); - return _importResultError(true, "cancelled", "Query cancelled"); + return importResultError(true, "cancelled", "Query cancelled"); } if (exec->isRowLimitComplete()) { @@ -282,7 +282,7 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ LOGS(_log, LOG_LVL_INFO, "UberJob ignoring, enough rows already " << "dataIgnored=" << dataIgnored); } - return _importResultError(false, "rowLimited", "Enough rows already"); + return importResultError(false, "rowLimited", "Enough rows already"); } LOGS(_log, LOG_LVL_TRACE, cName(__func__) << " fileSize=" << fileSize); @@ -290,52 +290,28 @@ json UberJob::importResultFile(string const& fileUrl, uint64_t rowCount, uint64_ bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_READY, getIdStr() + " " + fileUrl); if (!statusSet) { LOGS(_log, LOG_LVL_WARN, cName(__func__) << " setStatusFail could not set status to RESPONSE_READY"); - return _importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); + return importResultError(false, "setStatusFail", "could not set status to RESPONSE_READY"); } weak_ptr ujThis = weak_from_this(); - bool const limitSquash = exec->getLimitSquashApplies(); // fileCollectFunc will be put on the queue to run later. string const idStr = _idStr; - auto fileCollectFunc = [ujThis, fileUrl, fileSize, rowCount, idStr, limitSquash](util::CmdData*) { + auto fileCollectFunc = [ujThis, fileUrl, fileSize, rowCount, idStr](util::CmdData*) { auto ujPtr = ujThis.lock(); if (ujPtr == nullptr) { LOGS(_log, LOG_LVL_DEBUG, "UberJob::fileCollectFunction uberjob ptr is null " << idStr << " " << fileUrl); return; } - - // Limit collecting LIMIT queries to one(per UJ) at a time, but only those. - shared_ptr> limitSquashL; - if (limitSquash) { - limitSquashL.reset(new lock_guard(ujPtr->_mtxLimitSquash)); - } - MergeEndStatus flushStatus = ujPtr->getRespHandler()->flushHttp(fileUrl, fileSize); - LOGS(_log, LOG_LVL_TRACE, - ujPtr->cName(__func__) << "::fileCollectFunc success=" << flushStatus.success - << " contaminated=" << flushStatus.contaminated); - if (flushStatus.success) { - qdisp::CzarStats::get()->addTotalRowsRecv(rowCount); - qdisp::CzarStats::get()->addTotalBytesRecv(fileSize); - } else { - if (flushStatus.contaminated) { - // This would probably indicate malformed file+rowCount or writing the result table failed. - // If any merging happened, the result table is ruined. - LOGS(_log, LOG_LVL_ERROR, - ujPtr->cName(__func__) - << "::fileCollectFunc flushHttp failed after merging, results ruined."); - } else { - // Perhaps something went wrong with file collection, so it is worth trying the jobs again - // by abandoning this UberJob. - LOGS(_log, LOG_LVL_ERROR, - ujPtr->cName(__func__) << "::fileCollectFunc flushHttp failed, retrying Jobs."); - } - ujPtr->_importResultError(flushStatus.contaminated, "mergeError", "merging failed"); + auto exec = ujPtr->getExecutive(); + if (exec == nullptr) { + LOGS(_log, LOG_LVL_DEBUG, + "UberJob::fileCollectFunction exec ptr is null " << idStr << " " << fileUrl); + return; } - // At this point all data for this job have been read. - ujPtr->_importResultFinish(rowCount); + exec->collectFile(ujPtr, fileUrl, fileSize, rowCount, idStr); }; auto cmd = util::PriorityCommand::Ptr(new util::PriorityCommand(fileCollectFunc)); @@ -394,7 +370,7 @@ json UberJob::workerError(int errorCode, string const& errorMsg) { return _workerErrorFinish(deleteData, errType, ""); } -json UberJob::_importResultError(bool shouldCancel, string const& errorType, string const& note) { +json UberJob::importResultError(bool shouldCancel, string const& errorType, string const& note) { json jsRet = {{"success", 0}, {"errortype", errorType}, {"note", note}}; // In all cases, the worker should delete the file as this czar will not ask for it. @@ -425,32 +401,20 @@ json UberJob::_importResultError(bool shouldCancel, string const& errorType, str return jsRet; } -void UberJob::_importResultFinish(uint64_t resultRows) { +bool UberJob::importResultFinish() { LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " start"); - auto exec = _executive.lock(); - if (exec == nullptr) { - LOGS(_log, LOG_LVL_DEBUG, cName(__func__) << " executive is null"); - return; - } - /// If this is called, the file has been collected and the worker should delete it /// /// This function should call markComplete for all jobs in the uberjob /// and return a "success:1" json message to be sent to the worker. bool const statusSet = setStatusIfOk(qmeta::JobStatus::RESPONSE_DONE, getIdStr() + " _importResultFinish"); - if (!statusSet) { - LOGS(_log, LOG_LVL_ERROR, cName(__func__) << " failed to set status, squashing " << getIdStr()); - // Something has gone very wrong - exec->squash("UberJob::_importResultFinish couldn't set status"); - return; + if (statusSet) { + bool const success = true; + callMarkCompleteFunc(success); // sets status to COMPLETE } - - bool const success = true; - callMarkCompleteFunc(success); // sets status to COMPLETE - exec->addResultRows(resultRows); - exec->checkLimitRowComplete(); + return statusSet; } nlohmann::json UberJob::_workerErrorFinish(bool deleteData, std::string const& errorType, diff --git a/src/qdisp/UberJob.h b/src/qdisp/UberJob.h index ccf96f6ef..10fede387 100644 --- a/src/qdisp/UberJob.h +++ b/src/qdisp/UberJob.h @@ -116,6 +116,14 @@ class UberJob : public std::enable_shared_from_this { void setResultFileSize(uint64_t fileSize) { _resultFileSize = fileSize; } uint64_t getResultFileSize() { return _resultFileSize; } + /// Update UberJob status, return true if successful. + bool importResultFinish(); + + /// Import and error from trying to collect results. + /// TODO:UJ The strings for errorType should have a centralized location in the code - global or util + nlohmann::json importResultError(bool shouldCancel, std::string const& errorType, + std::string const& note); + std::ostream& dumpOS(std::ostream& os) const; std::string dump() const; friend std::ostream& operator<<(std::ostream& os, UberJob const& uj); @@ -137,14 +145,6 @@ class UberJob : public std::enable_shared_from_this { /// reassignment. The list of _jobs is cleared, so multiple calls of this should be harmless. void _unassignJobs(); - /// Import and error from trying to collect results. - /// TODO:UJ The strings for errorType should have a centralized location in the code - global or util - nlohmann::json _importResultError(bool shouldCancel, std::string const& errorType, - std::string const& note); - - /// Let the executive know that all Jobs in UberJob are complete. - void _importResultFinish(uint64_t resultRows); - /// Let the Executive know about errors while handling results. nlohmann::json _workerErrorFinish(bool successful, std::string const& errorType = std::string(), std::string const& note = std::string()); @@ -162,15 +162,8 @@ class UberJob : public std::enable_shared_from_this { qmeta::CzarId const _czarId; int const _rowLimit; ///< Number of rows in the query LIMIT clause. uint64_t _resultFileSize = 0; - std::string const _idStr; - /// This mutex is used to limit collecting result files to one at a time - /// but only when the executive will squash the query when the limit is reached. - /// This keeps data transfers (and temporary storage requirements) from - /// getting out of hand. - std::mutex _mtxLimitSquash; - // Map of workerData czar::CzarChunkMap::WorkerChunksData::Ptr _workerData; // TODO:UJ this may not be needed From 9fa1acd64c07c52c0d6aa50180e96f56e30c84c7 Mon Sep 17 00:00:00 2001 From: John Gates Date: Mon, 7 Jul 2025 11:56:20 -0700 Subject: [PATCH 11/11] Changed integration testing configuration. --- admin/local/docker/compose/docker-compose.yml | 6 +++--- .../templates/proxy/etc/qserv-czar.cnf.jinja | 17 +++++++++++++++++ src/cconfig/CzarConfig.h | 3 ++- src/czar/CzarRegistry.cc | 3 +-- src/mysql/CsvMemDisk.cc | 2 +- src/rproc/InfileMerger.h | 2 +- 6 files changed, 25 insertions(+), 8 deletions(-) diff --git a/admin/local/docker/compose/docker-compose.yml b/admin/local/docker/compose/docker-compose.yml index ebaa89ab4..66bb6fa91 100644 --- a/admin/local/docker/compose/docker-compose.yml +++ b/admin/local/docker/compose/docker-compose.yml @@ -36,7 +36,7 @@ volumes: volume_czar_xrootd: volume_czar_home: volume_czar_cfg: - volume_czar_tmp: + volume_czar_transfer: volume_czar_mariadb_data: volume_czar_mariadb_cfg: @@ -274,7 +274,7 @@ services: source: volume_czar_mariadb_run target: /qserv/mariadb/run - type: volume - source: volume_czar_tmp + source: volume_czar_transfer target: /tmp - << : *log-volume @@ -312,7 +312,7 @@ services: source: volume_czar_cfg target: /config-etc - type: volume - source: volume_czar_tmp + source: volume_czar_transfer target: /tmp - type: volume source: volume_czar_home diff --git a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja index 47ed4a72a..275cfd8b3 100644 --- a/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja +++ b/src/admin/templates/proxy/etc/qserv-czar.cnf.jinja @@ -25,12 +25,29 @@ port = {{ czar_db_port }} # Any table in resultdb that hasn't been updated in this many days is deleted. oldestResultKeptDays = 7 +# Either this should be changed to a high performance docker volume directory +# or /tmp should be mounted as a high performance docker volume directory +# to avoid using limited docker memory to store the contents. +transferDir = /tmp + # maximum number of connection retries to SQL databse (per connection attempt) maxsqlconnectionattempts = 10 # maximum user query result size in MB maxtablesize_mb = 5100 +# maximum number of MB of concurrent csv transfer files allowed to be kept in +# memory, after this point the will be temporarily written to disk. +# 0 is used for testing. 10000 is usually reasonable. +maxTransferMemMB = 0 + +# minimum number of MB for each csv transfer file to be kept in memory +# before possibly going to disk. +# 0 for testing, up to 10 should be reasonable. +transferMinMBInMem = 0 + + + # database connection for QMeta database [qmeta] diff --git a/src/cconfig/CzarConfig.h b/src/cconfig/CzarConfig.h index 785ec566f..fa8eac365 100644 --- a/src/cconfig/CzarConfig.h +++ b/src/cconfig/CzarConfig.h @@ -125,7 +125,8 @@ class CzarConfig { int getMaxTableSizeMB() const { return _maxTableSizeMB->getVal(); } int getMaxSqlConnectionAttempts() const { return _maxSqlConnectionAttempts->getVal(); } unsigned int getMaxTransferMemMB() const { return _resultMaxTransferMemMB->getVal(); } - /// Return the transfer directory, which defaults to /tmp, which is bad for performance. + /// Return the transfer directory. This is customizable to allow for a + /// high performance volume. std::string getTransferDir() const { return _resultTransferDir->getVal(); } /// Return the minimum amount of memory per UberJob to keep in memory. This much transfer diff --git a/src/czar/CzarRegistry.cc b/src/czar/CzarRegistry.cc index 03bfe3509..42369927e 100644 --- a/src/czar/CzarRegistry.cc +++ b/src/czar/CzarRegistry.cc @@ -80,14 +80,13 @@ void CzarRegistry::_registryUpdateLoop() { string const url = "http://" + _czarConfig->replicationRegistryHost() + ":" + to_string(_czarConfig->replicationRegistryPort()) + "/czar"; vector const headers = {"Content-Type: application/json"}; - string const fqdn = util::getCurrentHostFqdnBlocking(); json const request = json::object({{"instance_id", _czarConfig->replicationInstanceId()}, {"auth_key", _czarConfig->replicationAuthKey()}, {"czar", {{"name", _czarConfig->name()}, {"id", _czarConfig->id()}, {"management-port", _czarConfig->replicationHttpPort()}, - {"management-host-name", fqdn}}}}); + {"management-host-name", util::getCurrentHostFqdnBlocking()}}}}); string const requestContext = "Czar: '" + http::method2string(method) + "' request to '" + url + "'"; LOGS(_log, LOG_LVL_TRACE, __func__ << " czarPost url=" << url << " request=" << request.dump() << " headers=" << headers[0]); diff --git a/src/mysql/CsvMemDisk.cc b/src/mysql/CsvMemDisk.cc index d2c7eafef..4fc0502e6 100644 --- a/src/mysql/CsvMemDisk.cc +++ b/src/mysql/CsvMemDisk.cc @@ -118,7 +118,7 @@ bool CsvMemDisk::_mustWriteToTmpFile() { if (_writingToTmpFile) return true; auto memTrack = TransferTracker::get(); - // If too much memory is being used for transfers, star writing large transfers to files. + // If too much memory is being used for transfers, start writing large transfers to files. if (memTrack->getTotal() > memTrack->getMax()) { if (_records.size() > _minRecordsSize && _bytesRead > _minBytesInMem) { _writingToTmpFile = true; diff --git a/src/rproc/InfileMerger.h b/src/rproc/InfileMerger.h index 3e1890459..dc3a142fc 100644 --- a/src/rproc/InfileMerger.h +++ b/src/rproc/InfileMerger.h @@ -188,7 +188,7 @@ class InfileMerger { /// Variable to track result size. Each size_t const _maxResultTableSizeBytes; ///< Max result table size in bytes. size_t _totalResultSize = 0; ///< Size of result so far in bytes. - std::mutex _mtxResultSizeMtx; ///< Protects _perJobResultSize and _totalResultSize. + std::mutex _mtxResultSizeMtx; ///< Protects _totalResultSize. std::mutex _finalMergeMtx; ///< Protects mysql result tables };