From 739dc6739f085e097cb39dbabf6e24588722d8cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 13:31:02 +0200 Subject: [PATCH 01/31] Progress --- src/Interpreters/Cache/FileCache.cpp | 22 +++- src/Interpreters/Cache/FileCache.h | 5 +- src/Interpreters/Cache/FileCacheSettings.cpp | 8 +- src/Interpreters/Cache/FileCache_fwd.h | 3 +- src/Interpreters/Cache/FileSegment.cpp | 67 +++++----- src/Interpreters/Cache/Metadata.cpp | 128 ++++++++++++++++++- src/Interpreters/Cache/Metadata.h | 13 ++ 7 files changed, 201 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e6bc8b1f79b..65f8ecf7e89 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -49,9 +50,9 @@ FileCache::FileCache(const FileCacheSettings & settings) , allow_persistent_files(settings.do_not_evict_index_and_mark_files) , bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0) , delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms) + , boundary_alignment(settings.boundary_alignment) , log(&Poco::Logger::get("FileCache")) , metadata(settings.base_path) - , boundary_alignment(settings.boundary_alignment) { main_priority = std::make_unique(settings.max_size, settings.max_elements); @@ -124,7 +125,12 @@ void FileCache::initialize() is_initialized = true; - cleanup_task = Context::getGlobalContextInstance()->getSchedulePool().createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); + size_t num_threads=2; + for (size_t i = 0; i < num_threads; ++i) + download_threads.emplace_back([this] { metadata.downloadThreadFunc(); }); + + auto & schedule_pool = Context::getGlobalContextInstance()->getSchedulePool(); + cleanup_task = schedule_pool.createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); cleanup_task->activate(); cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); } @@ -412,7 +418,12 @@ FileSegmentsHolderPtr FileCache::set( } FileSegmentsHolderPtr -FileCache::getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings) +FileCache::getOrSet( + const Key & key, + size_t offset, + size_t size, + size_t file_size, + const CreateFileSegmentSettings & settings) { assertInitialized(); @@ -979,6 +990,10 @@ void FileCache::deactivateBackgroundOperations() { if (cleanup_task) cleanup_task->deactivate(); + + metadata.cancelDownload(); + for (auto & thread : download_threads) + thread.join(); } void FileCache::cleanup() @@ -1017,6 +1032,7 @@ FileSegmentsHolderPtr FileCache::getSnapshot() { for (const auto & [_, file_segment_metadata] : locked_key) file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment)); + return true; }); return std::make_unique(std::move(file_segments), /* complete_on_dtor */false); } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 71fc1722844..ed693b475ec 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -134,6 +134,7 @@ private: const bool allow_persistent_files; const size_t bypass_cache_threshold = 0; const size_t delayed_cleanup_interval_ms; + const size_t boundary_alignment; Poco::Logger * log; @@ -178,9 +179,9 @@ private: */ BackgroundSchedulePool::TaskHolder cleanup_task; - void assertInitialized() const; + std::vector download_threads; - size_t boundary_alignment; + void assertInitialized() const; void assertCacheCorrectness(); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 1fe51bf5f3e..d41aa8ffcb9 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -31,10 +31,9 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk Cache requires non-empty `path` field (cache base path) in config"); max_elements = config.getUInt64(config_prefix + ".max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS); + if (config.has(config_prefix + ".max_file_segment_size")) max_file_segment_size = parseWithSizeSuffix(config.getString(config_prefix + ".max_file_segment_size")); - else - max_file_segment_size = FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE; cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false); enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false); @@ -44,12 +43,11 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (config.has(config_prefix + ".bypass_cache_threashold")) bypass_cache_threashold = parseWithSizeSuffix(config.getString(config_prefix + ".bypass_cache_threashold")); - else - bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", true); - boundary_alignment = config.getUInt64(config_prefix + ".boundary_alignment", DBMS_DEFAULT_BUFFER_SIZE); + if (config.has(config_prefix + ".boundary_alignment")) + boundary_alignment = parseWithSizeSuffix(config.getString(config_prefix + ".boundary_alignment")); delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS); } diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 01f518d0c4e..c791d08ed4f 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -4,7 +4,8 @@ namespace DB { -static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 8 * 1024 * 1024; +static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi +static constexpr int FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 7b82c58080c..f95379ba07f 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -214,8 +214,10 @@ void FileSegment::resetDownloadingStateUnlocked(const FileSegmentGuard::Lock & l /// range().size() can equal 0 in case of write-though cache. if (!is_unbound && current_downloaded_size != 0 && current_downloaded_size == range().size()) setDownloadedUnlocked(lock); - else + else if (current_downloaded_size) setDownloadState(State::PARTIALLY_DOWNLOADED, lock); + else + setDownloadState(State::EMPTY, lock); } void FileSegment::resetDownloader() @@ -280,22 +282,9 @@ void FileSegment::resetRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - auto locked_key = lockKeyMetadata(false); - if (!locked_key) - { - assert(isDetached()); + if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) return std::move(remote_file_reader); - } - - auto segment_lock = segment_guard.lock(); - - assert(download_state != State::DETACHED); - - bool is_last_holder = locked_key->isLastOwnerOfFileSegment(offset()); - if (!downloader_id.empty() || !is_last_holder) - return nullptr; - - return std::move(remote_file_reader); + return nullptr; } void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) @@ -607,22 +596,17 @@ void FileSegment::complete() resetDownloaderUnlocked(segment_lock); } - if (is_downloader || is_last_holder) + auto remove_from_cache = [&, this]() { - if (cache_writer) - { - cache_writer->finalize(); - cache_writer.reset(); - } - remote_file_reader.reset(); - } + LOG_TEST(log, "Remove file segment {} (nothing downloaded)", range().toString()); + locked_key->removeFileSegment(offset(), segment_lock); + setDetachedState(segment_lock); + }; if (segment_kind == FileSegmentKind::Temporary && is_last_holder) { LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); - detach(segment_lock, *locked_key); - setDownloadState(State::DETACHED, segment_lock); - locked_key->removeFileSegment(offset(), segment_lock); + remove_from_cache(); return; } @@ -633,6 +617,7 @@ void FileSegment::complete() chassert(current_downloaded_size == range().size()); chassert(current_downloaded_size == fs::file_size(getPathInLocalCache())); chassert(!cache_writer); + chassert(!remote_file_reader); break; } case State::DOWNLOADING: @@ -640,8 +625,24 @@ void FileSegment::complete() chassert(!is_last_holder); break; } - case State::EMPTY: case State::PARTIALLY_DOWNLOADED: + { + if (is_last_holder) + { + LOG_TEST( + log, "Submitted file segment for background download " + "(having {}/{})", downloaded_size, range().size()); + + locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background. + } + break; + } + case State::EMPTY: + { + if (is_last_holder) + remove_from_cache(); + break; + } case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { chassert(current_downloaded_size != range().size()); @@ -650,10 +651,9 @@ void FileSegment::complete() { if (current_downloaded_size == 0) { - LOG_TEST(log, "Remove file segment {} (nothing downloaded)", range().toString()); - locked_key->removeFileSegment(offset(), segment_lock); + remove_from_cache(); } - else + else if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) { LOG_TEST(log, "Resize file segment {} to downloaded: {}", range().toString(), current_downloaded_size); @@ -672,9 +672,8 @@ void FileSegment::complete() /// We mark current file segment with state DETACHED, even though the data is still in cache /// (but a separate file segment) because is_last_holder is satisfied, so it does not matter. + setDetachedState(segment_lock); } - - setDetachedState(segment_lock); } break; } @@ -844,6 +843,8 @@ void FileSegment::setDetachedState(const FileSegmentGuard::Lock & lock) setDownloadState(State::DETACHED, lock); key_metadata.reset(); cache = nullptr; + cache_writer.reset(); + remote_file_reader.reset(); } void FileSegment::detach(const FileSegmentGuard::Lock & lock, const LockedKey &) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c9a23d1c785..2a3803eb3d7 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -46,10 +46,12 @@ KeyMetadata::KeyMetadata( const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, + DownloadQueue & download_queue_, bool created_base_directory_) : key(key_) , key_path(key_path_) , cleanup_queue(cleanup_queue_) + , download_queue(download_queue_) , created_base_directory(created_base_directory_) { if (created_base_directory) @@ -123,6 +125,7 @@ private: CacheMetadata::CacheMetadata(const std::string & path_) : path(path_) , cleanup_queue(std::make_unique()) + , download_queue(std::make_unique()) , log(&Poco::Logger::get("CacheMetadata")) { } @@ -175,7 +178,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( it = emplace( key, std::make_shared( - key, getPathForKey(key), *cleanup_queue, is_initial_load)).first; + key, getPathForKey(key), *cleanup_queue, *download_queue, is_initial_load)).first; } key_metadata = it->second; @@ -293,6 +296,121 @@ void CacheMetadata::doCleanup() } } +class DownloadQueue +{ +friend struct CacheMetadata; +public: + void add(std::weak_ptr file_segment) + { + { + std::lock_guard lock(mutex); + queue.push(file_segment); + } + cv.notify_one(); + } + +private: + void cancel() + { + std::lock_guard lock(mutex); + cancelled = true; + } + + std::mutex mutex; + std::condition_variable cv; + std::queue> queue; + bool cancelled = false; +}; + +void CacheMetadata::downloadThreadFunc() +{ + std::optional> memory; + while (true) + { + std::weak_ptr file_segment_weak; + { + std::unique_lock lock(download_queue->mutex); + + if (download_queue->cancelled) + return; + + if (download_queue->queue.empty()) + { + download_queue->cv.wait(lock); + continue; + } + + file_segment_weak = download_queue->queue.front(); + download_queue->queue.pop(); + } + + FileSegmentsHolderPtr holder; + { + auto file_segment = file_segment_weak.lock(); + if (!file_segment + || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) + continue; + + auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); + if (!lock) + continue; + + holder = std::make_unique(FileSegments{file_segment}); + } + + auto & file_segment = holder->front(); + chassert(file_segment.assertCorrectness()); + + if (file_segment.getOrSetDownloader() != FileSegment::getCallerId()) + continue; + + LOG_TRACE(log, "Downloading file segment: {}", file_segment.getInfoForLog()); + + auto reader = file_segment.getRemoteFileReader(); + + /// If remote_fs_read_method == 'threadpool', + /// reader iteself does not allocate the buffer, but uses the buffer passed to it. + /// So will need to allocate a buffer here as well. + if (reader->buffer().begin() == nullptr) + { + if (!memory) + memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); + reader->set(memory->data(), memory->size()); + } + + size_t offset = file_segment.getCurrentWriteOffset(false); + while (!reader->eof()) + { + auto size = reader->available(); + + if (!file_segment.reserve(size)) + return; + + try + { + file_segment.write(reader->position(), size, offset); + offset += size; + } + catch (ErrnoException & e) + { + int code = e.getErrno(); + if (code == /* No space left on device */28 || code == /* Quota exceeded */122) + { + LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); + continue; + } + throw; + } + } + } +} + +void CacheMetadata::cancelDownload() +{ + download_queue->cancel(); + download_queue->cv.notify_all(); +} + LockedKey::LockedKey(std::shared_ptr key_metadata_) : key_metadata(key_metadata_) , lock(key_metadata->guard.lock()) @@ -426,6 +544,14 @@ void LockedKey::shrinkFileSegmentToDownloadedSize( chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); } +void LockedKey::addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &) +{ + auto it = key_metadata->find(offset); + if (it == key_metadata->end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset); + key_metadata->download_queue.add(it->second->file_segment); +} + std::shared_ptr LockedKey::getByOffset(size_t offset) const { auto it = key_metadata->find(offset); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 3fd6176f201..f96243c3f1f 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -8,8 +8,12 @@ namespace DB { + class CleanupQueue; using CleanupQueuePtr = std::shared_ptr; +class DownloadQueue; +using DownloadQueuePtr = std::shared_ptr; +using FileSegmentsHolderPtr = std::unique_ptr; struct FileSegmentMetadata : private boost::noncopyable @@ -44,6 +48,7 @@ struct KeyMetadata : public std::map, const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, + DownloadQueue & download_queue_, bool created_base_directory_ = false); enum class KeyState @@ -69,6 +74,7 @@ private: KeyState key_state = KeyState::ACTIVE; KeyGuard guard; CleanupQueue & cleanup_queue; + DownloadQueue & download_queue; std::atomic created_base_directory = false; }; @@ -109,10 +115,15 @@ public: void doCleanup(); + void downloadThreadFunc(); + + void cancelDownload(); + private: const std::string path; /// Cache base path CacheMetadataGuard guard; const CleanupQueuePtr cleanup_queue; + const DownloadQueuePtr download_queue; Poco::Logger * log; }; @@ -159,6 +170,8 @@ struct LockedKey : private boost::noncopyable void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &); + void addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &); + bool isLastOwnerOfFileSegment(size_t offset) const; void removeFromCleanupQueue(); From 96fb7f04cbc94c625229afd2958cea056c563920 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 15:11:43 +0200 Subject: [PATCH 02/31] Finish download of partially downloaded file segments in the background instead of resizing --- src/Common/CurrentMetrics.cpp | 1 + src/Interpreters/Cache/FileCache.cpp | 38 ++--- src/Interpreters/Cache/FileCache.h | 1 + src/Interpreters/Cache/FileCacheSettings.cpp | 3 + src/Interpreters/Cache/FileCacheSettings.h | 3 +- src/Interpreters/Cache/FileCache_fwd.h | 1 + src/Interpreters/Cache/FileSegment.cpp | 20 ++- src/Interpreters/Cache/Metadata.cpp | 154 ++++++++++++------ src/Interpreters/Cache/Metadata.h | 2 + ...02789_filesystem_cache_alignment.reference | 7 + .../02789_filesystem_cache_alignment.sh | 125 ++++++++++++++ 11 files changed, 264 insertions(+), 91 deletions(-) create mode 100644 tests/queries/0_stateless/02789_filesystem_cache_alignment.reference create mode 100755 tests/queries/0_stateless/02789_filesystem_cache_alignment.sh diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 61725d079bf..b479eac3021 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -188,6 +188,7 @@ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ + M(FilesystemCacheDownloadQueueElements, "Filesystem cache elements in download queue") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ M(S3Requests, "S3 requests") \ M(KeeperAliveConnections, "Number of alive connections") \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 65f8ecf7e89..c090d3accd4 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -51,6 +51,7 @@ FileCache::FileCache(const FileCacheSettings & settings) , bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0) , delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms) , boundary_alignment(settings.boundary_alignment) + , background_download_threads(settings.background_download_threads) , log(&Poco::Logger::get("FileCache")) , metadata(settings.base_path) { @@ -125,12 +126,10 @@ void FileCache::initialize() is_initialized = true; - size_t num_threads=2; - for (size_t i = 0; i < num_threads; ++i) + for (size_t i = 0; i < background_download_threads; ++i) download_threads.emplace_back([this] { metadata.downloadThreadFunc(); }); - auto & schedule_pool = Context::getGlobalContextInstance()->getSchedulePool(); - cleanup_task = schedule_pool.createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); + cleanup_task = Context::getGlobalContextInstance()->getSchedulePool().createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); cleanup_task->activate(); cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); } @@ -644,27 +643,14 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) if (releasable) { - auto segment = segment_metadata->file_segment; - if (segment->state() == FileSegment::State::DOWNLOADED) - { - const auto & key = segment->key(); + const auto & key = segment_metadata->file_segment->key(); + auto it = to_delete.find(key); + if (it == to_delete.end()) + it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; + it->second.add(segment_metadata); - auto it = to_delete.find(key); - if (it == to_delete.end()) - it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; - it->second.add(segment_metadata); - - freeable_space += segment_metadata->size(); - freeable_count += 1; - - return PriorityIterationResult::CONTINUE; - } - - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->getDownloadedSize(false)); - - locked_key.removeFileSegment(segment->offset(), segment->lock()); - return PriorityIterationResult::REMOVE_AND_CONTINUE; + freeable_space += segment_metadata->size(); + freeable_count += 1; } return PriorityIterationResult::CONTINUE; }; @@ -1005,10 +991,6 @@ void FileCache::cleanupThreadFunc() { try { -#ifdef ABORT_ON_LOGICAL_ERROR - assertCacheCorrectness(); -#endif - cleanup(); } catch (...) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index ed693b475ec..931cecf607b 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -135,6 +135,7 @@ private: const size_t bypass_cache_threshold = 0; const size_t delayed_cleanup_interval_ms; const size_t boundary_alignment; + const size_t background_download_threads; Poco::Logger * log; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index d41aa8ffcb9..bc6e641c869 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -49,6 +49,9 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (config.has(config_prefix + ".boundary_alignment")) boundary_alignment = parseWithSizeSuffix(config.getString(config_prefix + ".boundary_alignment")); + if (config.has(config_prefix + ".background_download_threads")) + background_download_threads = config.getUInt(config_prefix + ".background_download_threads"); + delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS); } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index eeb2a02c131..fcc5c02c52e 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -28,7 +28,8 @@ struct FileCacheSettings size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS; - size_t boundary_alignment = DBMS_DEFAULT_BUFFER_SIZE; + size_t boundary_alignment = FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE; + size_t background_download_threads = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS; void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); }; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index c791d08ed4f..902a6ff42d0 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -6,6 +6,7 @@ namespace DB static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi static constexpr int FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE = 4 * 1024 * 1024; /// 4Mi +static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index f95379ba07f..5dd35a720c1 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -282,8 +282,11 @@ void FileSegment::resetRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + if (isCompleted(false) + || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + { return std::move(remote_file_reader); + } return nullptr; } @@ -598,7 +601,6 @@ void FileSegment::complete() auto remove_from_cache = [&, this]() { - LOG_TEST(log, "Remove file segment {} (nothing downloaded)", range().toString()); locked_key->removeFileSegment(offset(), segment_lock); setDetachedState(segment_lock); }; @@ -625,6 +627,12 @@ void FileSegment::complete() chassert(!is_last_holder); break; } + case State::EMPTY: + { + if (is_last_holder) + remove_from_cache(); + break; + } case State::PARTIALLY_DOWNLOADED: { if (is_last_holder) @@ -637,12 +645,6 @@ void FileSegment::complete() } break; } - case State::EMPTY: - { - if (is_last_holder) - remove_from_cache(); - break; - } case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { chassert(current_downloaded_size != range().size()); @@ -653,7 +655,7 @@ void FileSegment::complete() { remove_from_cache(); } - else if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + else { LOG_TEST(log, "Resize file segment {} to downloaded: {}", range().toString(), current_downloaded_size); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 2a3803eb3d7..c3f428c3e08 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -1,11 +1,17 @@ #include #include #include +#include "Common/Exception.h" #include #include namespace fs = std::filesystem; +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheDownloadQueueElements; +} + namespace DB { @@ -256,6 +262,7 @@ void CacheMetadata::doCleanup() continue; } + chassert(it->second->empty()); locked_metadata->markAsRemoved(); erase(it); LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); @@ -306,6 +313,8 @@ public: std::lock_guard lock(mutex); queue.push(file_segment); } + + CurrentMetrics::add(CurrentMetrics::FilesystemCacheDownloadQueueElements); cv.notify_one(); } @@ -324,10 +333,10 @@ private: void CacheMetadata::downloadThreadFunc() { - std::optional> memory; while (true) { std::weak_ptr file_segment_weak; + { std::unique_lock lock(download_queue->mutex); @@ -344,67 +353,106 @@ void CacheMetadata::downloadThreadFunc() download_queue->queue.pop(); } + CurrentMetrics::sub(CurrentMetrics::FilesystemCacheDownloadQueueElements); + FileSegmentsHolderPtr holder; + try { - auto file_segment = file_segment_weak.lock(); - if (!file_segment - || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) - continue; - - auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); - if (!lock) - continue; - - holder = std::make_unique(FileSegments{file_segment}); - } - - auto & file_segment = holder->front(); - chassert(file_segment.assertCorrectness()); - - if (file_segment.getOrSetDownloader() != FileSegment::getCallerId()) - continue; - - LOG_TRACE(log, "Downloading file segment: {}", file_segment.getInfoForLog()); - - auto reader = file_segment.getRemoteFileReader(); - - /// If remote_fs_read_method == 'threadpool', - /// reader iteself does not allocate the buffer, but uses the buffer passed to it. - /// So will need to allocate a buffer here as well. - if (reader->buffer().begin() == nullptr) - { - if (!memory) - memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); - reader->set(memory->data(), memory->size()); - } - - size_t offset = file_segment.getCurrentWriteOffset(false); - while (!reader->eof()) - { - auto size = reader->available(); - - if (!file_segment.reserve(size)) - return; - - try { - file_segment.write(reader->position(), size, offset); - offset += size; - } - catch (ErrnoException & e) - { - int code = e.getErrno(); - if (code == /* No space left on device */28 || code == /* Quota exceeded */122) - { - LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); + auto file_segment = file_segment_weak.lock(); + if (!file_segment + || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) continue; - } - throw; + + auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); + if (!lock) + continue; + + holder = std::make_unique(FileSegments{file_segment}); + } + + downloadImpl(holder->front()); + } + catch (...) + { + if (holder) + { + const auto & file_segment = holder->front(); + LOG_ERROR( + log, "Error during background download of {}:{} ({}): {}", + file_segment.key(), file_segment.offset(), + file_segment.getInfoForLog(), getCurrentExceptionMessage(true)); + } + else + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); } } } } +void CacheMetadata::downloadImpl(FileSegment & file_segment) +{ + chassert(file_segment.assertCorrectness()); + + if (file_segment.getOrSetDownloader() != FileSegment::getCallerId()) + return; + + LOG_TEST( + log, "Downloading {} bytes for file segment {}", + file_segment.range().size() - file_segment.getDownloadedSize(false), file_segment.getInfoForLog()); + + auto reader = file_segment.getRemoteFileReader(); + + /// If remote_fs_read_method == 'threadpool', + /// reader iteself bever owns/allocates the buffer. + std::optional> memory; + if (reader->internalBuffer().empty()) + { + memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); + reader->set(memory->data(), memory->size()); + } + + size_t offset = file_segment.getCurrentWriteOffset(false); + if (offset != static_cast(reader->getPosition())) + reader->seek(offset, SEEK_SET); + + while (!reader->eof()) + { + auto size = reader->available(); + + if (!file_segment.reserve(size)) + { + LOG_TEST( + log, "Failed to reserve space during background download " + "for {}:{} (downloaded size: {}/{})", + file_segment.key(), file_segment.offset(), + file_segment.getDownloadedSize(false), file_segment.range().size()); + return; + } + + try + { + file_segment.write(reader->position(), size, offset); + offset += size; + reader->position() += size; + } + catch (ErrnoException & e) + { + int code = e.getErrno(); + if (code == /* No space left on device */28 || code == /* Quota exceeded */122) + { + LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); + return; + } + throw; + } + } + + LOG_TEST(log, "Downloaded file segment: {}", file_segment.getInfoForLog()); +} + void CacheMetadata::cancelDownload() { download_queue->cancel(); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index f96243c3f1f..7e8fdef6641 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -125,6 +125,8 @@ private: const CleanupQueuePtr cleanup_queue; const DownloadQueuePtr download_queue; Poco::Logger * log; + + void downloadImpl(FileSegment & file_segment); }; diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.reference b/tests/queries/0_stateless/02789_filesystem_cache_alignment.reference new file mode 100644 index 00000000000..70e1fca6a65 --- /dev/null +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.reference @@ -0,0 +1,7 @@ +0 +OK +OK +0 +0 +OK +OK diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh new file mode 100755 index 00000000000..a8bb4440a4c --- /dev/null +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -0,0 +1,125 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() +ORDER BY a +SETTINGS disk = disk(type = cache, + max_size = '1Gi', + max_file_segment_size = '40Mi', + boundary_alignment = '20Mi', + path = '$CLICKHOUSE_TEST_UNIQUE_NAME', + disk = 's3disk'); + +INSERT INTO test SELECT number, randomString(100) FROM numbers(1000000); +" + +QUERY_ID=$RANDOM +$CLICKHOUSE_CLIENT --query_id "$QUERY_ID" -nm -q " +SET enable_filesystem_cache_log = 1; +SYSTEM DROP FILESYSTEM CACHE; +SELECT * FROM test WHERE NOT ignore() LIMIT 1 FORMAT Null; +SYSTEM FLUSH LOGS; +" + +query=" +SELECT cache_path, file_size, + tupleElement(file_segment_range, 2) - tupleElement(file_segment_range, 1) + 1 as file_segment_size, + formatReadableSize(file_size) as formatted_file_size, + formatReadableSize(file_segment_size) as formatted_file_segment_size, + tupleElement(file_segment_range, 2) as end_offset +FROM ( + SELECT arrayJoin(cache_paths) AS cache_path, + local_path, + remote_path, + size as file_size + FROM system.remote_data_paths + WHERE endsWith(local_path, '.bin') +) AS data_paths +INNER JOIN system.filesystem_cache_log AS cache_log +ON data_paths.remote_path = cache_log.source_file_path +WHERE query_id = '$QUERY_ID' " + +# File segments cannot be less that 20Mi, +# except for last file segment in a file or if file size is less. +$CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query) +WHERE file_segment_size < file_size +AND end_offset + 1 != file_size +AND file_segment_size < 20 * 1024 * 1024; +" + +all=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query) +WHERE file_segment_size < file_size AND end_offset + 1 != file_size; +") +#echo $all + +if [ "$all" -gt "10" ]; then + echo "OK" +else + echo "FAIL" +fi + +count=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query) +WHERE file_segment_size < file_size +AND end_offset + 1 != file_size +AND formatted_file_segment_size in ('20.00 MiB', '40.00 MiB') +") + +if [ "$count" = "$all" ]; then + echo "OK" +else + echo "FAIL" +fi + +query2=" +SELECT * +FROM (SELECT * FROM ($query)) AS cache_log +INNER JOIN system.filesystem_cache AS cache +ON cache_log.cache_path = cache.cache_path " + +$CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_range_begin - file_segment_range_end + 1 < file_size +AND file_segment_range_end + 1 != file_size +AND downloaded_size < 20 * 1024 * 1024; +" + +$CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_range_begin - file_segment_range_end + 1 < file_size +AND file_segment_range_end + 1 != file_size +AND formatReadableSize(downloaded_size) not in ('20.00 MiB', '40.00 MiB'); +" + +all=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_size < file_size AND file_segment_range_end + 1 != file_size; +") + +if [ "$all" -gt "10" ]; then + echo "OK" +else + echo "FAIL" +fi + +count2=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_range_begin - file_segment_range_end + 1 < file_size +AND file_segment_range_end + 1 != file_size +AND formatReadableSize(downloaded_size) in ('20.00 MiB', '40.00 MiB'); +") + +if [ "$count2" = "$all" ]; then + echo "OK" +else + echo "FAIL" +fi From 0aa363f5254455ea3a351c9fcc9adc7c99701c53 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 23:13:09 +0200 Subject: [PATCH 03/31] Fix style check --- src/Interpreters/Cache/Metadata.cpp | 2 +- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c3f428c3e08..b0009c6abc1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -406,7 +406,7 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment) auto reader = file_segment.getRemoteFileReader(); /// If remote_fs_read_method == 'threadpool', - /// reader iteself bever owns/allocates the buffer. + /// reader itself never owns/allocates the buffer. std::optional> memory; if (reader->internalBuffer().empty()) { diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index a8bb4440a4c..ca459153c03 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -15,7 +15,7 @@ SETTINGS disk = disk(type = cache, max_file_segment_size = '40Mi', boundary_alignment = '20Mi', path = '$CLICKHOUSE_TEST_UNIQUE_NAME', - disk = 's3disk'); + disk = 's3_disk'); INSERT INTO test SELECT number, randomString(100) FROM numbers(1000000); " From 8c610275c2e271cd3a506d207fd9cef938fcd258 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 23:30:21 +0200 Subject: [PATCH 04/31] Review fixes --- src/Interpreters/Cache/FileCache.cpp | 1 - src/Interpreters/Cache/FileCacheSettings.h | 2 +- src/Interpreters/Cache/FileCache_fwd.h | 2 +- src/Interpreters/Cache/Metadata.cpp | 17 ++++++++++------- src/Interpreters/Cache/Metadata.h | 2 +- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index c090d3accd4..3b5ae9acb1b 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1014,7 +1014,6 @@ FileSegmentsHolderPtr FileCache::getSnapshot() { for (const auto & [_, file_segment_metadata] : locked_key) file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment)); - return true; }); return std::make_unique(std::move(file_segments), /* complete_on_dtor */false); } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index fcc5c02c52e..6820aff61e3 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -28,7 +28,7 @@ struct FileCacheSettings size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS; - size_t boundary_alignment = FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE; + size_t boundary_alignment = FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT; size_t background_download_threads = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS; void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 902a6ff42d0..4d4a1c1429c 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -5,7 +5,7 @@ namespace DB { static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi -static constexpr int FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE = 4 * 1024 * 1024; /// 4Mi +static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index b0009c6abc1..f94b8ae4db2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -321,8 +321,11 @@ public: private: void cancel() { - std::lock_guard lock(mutex); - cancelled = true; + { + std::lock_guard lock(mutex); + cancelled = true; + } + cv.notify_all(); } std::mutex mutex; @@ -333,6 +336,7 @@ private: void CacheMetadata::downloadThreadFunc() { + std::optional> memory; while (true) { std::weak_ptr file_segment_weak; @@ -371,7 +375,7 @@ void CacheMetadata::downloadThreadFunc() holder = std::make_unique(FileSegments{file_segment}); } - downloadImpl(holder->front()); + downloadImpl(holder->front(), memory); } catch (...) { @@ -392,7 +396,7 @@ void CacheMetadata::downloadThreadFunc() } } -void CacheMetadata::downloadImpl(FileSegment & file_segment) +void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional> & memory) { chassert(file_segment.assertCorrectness()); @@ -407,10 +411,10 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment) /// If remote_fs_read_method == 'threadpool', /// reader itself never owns/allocates the buffer. - std::optional> memory; if (reader->internalBuffer().empty()) { - memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); + if (!memory) + memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); reader->set(memory->data(), memory->size()); } @@ -456,7 +460,6 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment) void CacheMetadata::cancelDownload() { download_queue->cancel(); - download_queue->cv.notify_all(); } LockedKey::LockedKey(std::shared_ptr key_metadata_) diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 7e8fdef6641..f9f7dfccaa5 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -126,7 +126,7 @@ private: const DownloadQueuePtr download_queue; Poco::Logger * log; - void downloadImpl(FileSegment & file_segment); + void downloadImpl(FileSegment & file_segment, std::optional> & memory); }; From 0023bf69cbaf2ec756bca883e3bb6e9338e574de Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Jun 2023 12:22:50 +0200 Subject: [PATCH 05/31] Fix --- src/Interpreters/Cache/FileSegment.cpp | 18 ++++++++++--- src/Interpreters/Cache/Metadata.cpp | 3 +++ .../InterpreterDescribeCacheQuery.cpp | 27 ++++++++++++------- .../02344_describe_cache.reference | 4 +-- 4 files changed, 37 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 5dd35a720c1..4cc5e4307ad 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -635,13 +635,23 @@ void FileSegment::complete() } case State::PARTIALLY_DOWNLOADED: { + chassert(current_downloaded_size > 0); + if (is_last_holder) { - LOG_TEST( - log, "Submitted file segment for background download " - "(having {}/{})", downloaded_size, range().size()); + if (remote_file_reader) + { + LOG_TEST( + log, "Submitting file segment for background download " + "(having {}/{})", downloaded_size, range().size()); - locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background. + locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background. + } + else + { + locked_key->shrinkFileSegmentToDownloadedSize(offset(), segment_lock); + setDetachedState(segment_lock); /// See comment below. + } } break; } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f94b8ae4db2..fd590315ab1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -403,6 +403,9 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional(), "max_size"}, ColumnWithTypeAndName{std::make_shared(), "max_elements"}, ColumnWithTypeAndName{std::make_shared(), "max_file_segment_size"}, + ColumnWithTypeAndName{std::make_shared(), "boundary_alignment"}, ColumnWithTypeAndName{std::make_shared>(), "cache_on_write_operations"}, ColumnWithTypeAndName{std::make_shared>(), "cache_hits_threshold"}, ColumnWithTypeAndName{std::make_shared(), "current_size"}, ColumnWithTypeAndName{std::make_shared(), "current_elements"}, ColumnWithTypeAndName{std::make_shared(), "path"}, ColumnWithTypeAndName{std::make_shared>(), "do_not_evict_index_and_mark_files"}, + ColumnWithTypeAndName{std::make_shared>(), "delayed_cleanup_interval_ms"}, + ColumnWithTypeAndName{std::make_shared>(), "background_download_threads"}, + ColumnWithTypeAndName{std::make_shared>(), "enable_bypass_cache_with_threshold"}, }; return Block(columns); } @@ -41,15 +45,20 @@ BlockIO InterpreterDescribeCacheQuery::execute() const auto & settings = cache_data.settings; const auto & cache = cache_data.cache; - res_columns[0]->insert(settings.max_size); - res_columns[1]->insert(settings.max_elements); - res_columns[2]->insert(settings.max_file_segment_size); - res_columns[3]->insert(settings.cache_on_write_operations); - res_columns[4]->insert(settings.cache_hits_threshold); - res_columns[5]->insert(cache->getUsedCacheSize()); - res_columns[6]->insert(cache->getFileSegmentsNum()); - res_columns[7]->insert(cache->getBasePath()); - res_columns[8]->insert(settings.do_not_evict_index_and_mark_files); + size_t i = 0; + res_columns[i++]->insert(settings.max_size); + res_columns[i++]->insert(settings.max_elements); + res_columns[i++]->insert(settings.max_file_segment_size); + res_columns[i++]->insert(settings.boundary_alignment); + res_columns[i++]->insert(settings.cache_on_write_operations); + res_columns[i++]->insert(settings.cache_hits_threshold); + res_columns[i++]->insert(cache->getUsedCacheSize()); + res_columns[i++]->insert(cache->getFileSegmentsNum()); + res_columns[i++]->insert(cache->getBasePath()); + res_columns[i++]->insert(settings.do_not_evict_index_and_mark_files); + res_columns[i++]->insert(settings.delayed_cleanup_interval_ms); + res_columns[i++]->insert(settings.background_download_threads); + res_columns[i++]->insert(settings.enable_bypass_cache_with_threashold); BlockIO res; size_t num_rows = res_columns[0]->size(); diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index d8a2ffab1fa..7496f754937 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 -134217728 10000000 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 60000 2 0 +134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 60000 2 0 From b8a4a784debcfb4e739b2368bc5611885ffa2a51 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Jun 2023 14:22:13 +0200 Subject: [PATCH 06/31] Fix unit test, add check --- src/Interpreters/Cache/Metadata.cpp | 8 ++++++++ src/Interpreters/tests/gtest_lru_file_cache.cpp | 4 ++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index fd590315ab1..987361c0537 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -412,6 +412,14 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optionalinternalBuffer().empty()) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 8c8e715ce92..d8baf1c0b45 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -533,8 +533,8 @@ TEST_F(FileCacheTest, get) cv.notify_one(); file_segment2.wait(file_segment2.range().left); - ASSERT_TRUE(file_segment2.state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); - ASSERT_TRUE(file_segment2.getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_EQ(file_segment2.state(), DB::FileSegment::State::EMPTY); + ASSERT_EQ(file_segment2.getOrSetDownloader(), DB::FileSegment::getCallerId()); download(file_segment2); }); From ddc95796d3da99e0de3cf8b665c319f5305f39fe Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 16 Jun 2023 13:45:29 +0200 Subject: [PATCH 07/31] Fix tests --- tests/queries/0_stateless/02344_describe_cache.reference | 4 ++-- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7496f754937..97bcf240a16 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 60000 2 0 -134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 60000 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 100 2 0 +134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 100 2 0 diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index ca459153c03..b9f9548a7b3 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest, no-parallel, no-random-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 12f979dd2bcc66ba396f4a3eed6e03d280661c49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 17 Jun 2023 02:08:11 +0200 Subject: [PATCH 08/31] Update .reference --- tests/queries/0_stateless/02344_describe_cache.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 97bcf240a16..6f445dc13e7 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 100 2 0 -134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 100 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 100 2 0 +134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 100 2 0 From 7df815d2806a9154a4f7010590952a647be97edd Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 20 Jun 2023 13:33:16 +0200 Subject: [PATCH 09/31] Fix --- src/Interpreters/Cache/Metadata.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index d659a58fdaf..ed2174213e1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -400,6 +400,10 @@ void CacheMetadata::downloadThreadFunc() if (!lock) continue; + auto file_segment_metadata = lock->tryGetByOffset(file_segment->offset()); + if (!file_segment_metadata || file_segment_metadata->evicting()) + continue; + holder = std::make_unique(FileSegments{file_segment}); } From 44377b01809dd4e85ca66d03c0b83a6f9da14ef9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 20 Jun 2023 13:36:11 +0200 Subject: [PATCH 10/31] Rename --- src/Interpreters/Cache/Metadata.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index ed2174213e1..76dd5172929 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -396,11 +396,11 @@ void CacheMetadata::downloadThreadFunc() || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) continue; - auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); - if (!lock) + auto locked_key = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); + if (!locked_key) continue; - auto file_segment_metadata = lock->tryGetByOffset(file_segment->offset()); + auto file_segment_metadata = locked_key->tryGetByOffset(file_segment->offset()); if (!file_segment_metadata || file_segment_metadata->evicting()) continue; From b8085dfa0e63b88132abcaecee5084936515a6d6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Jun 2023 00:20:40 +0200 Subject: [PATCH 11/31] Fix bad merge --- src/Interpreters/Cache/FileSegment.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index de8ede69848..daae9b447a4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -305,8 +305,9 @@ void FileSegment::resetRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - if (isCompleted(false) - || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + auto lock = lockFileSegment(); + if (remote_file_reader && (download_state == State::DOWNLOADED + || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION)) { return std::move(remote_file_reader); } From 0b6d367bdbe73e1ea9b1f179f315cba2185ffe94 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Jun 2023 17:14:49 +0000 Subject: [PATCH 12/31] Parts mover: lock between getActiveContainingPart and swapActivePart --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++-- src/Storages/MergeTree/MergeTreeData.h | 3 ++- src/Storages/MergeTree/MergeTreePartsMover.cpp | 7 +++++-- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9c3a7f66ae..5e17559acc0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4529,9 +4529,8 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart( } -void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) +void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy, DataPartsLock &) { - auto lock = lockParts(); for (auto original_active_part : getDataPartsStateRange(DataPartState::Active)) // NOLINT (copy is intended) { if (part_copy->name == original_active_part->name) @@ -4587,6 +4586,12 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & return getActiveContainingPart(part_info); } +MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & part_name, DataPartsLock & lock) const +{ + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + return getActiveContainingPart(part_info, DataPartState::Active, lock); +} + MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const { return getVisibleDataPartsVectorInPartition(local_context->getCurrentTransaction().get(), partition_id); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b27392b355b..c821a436a76 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -504,12 +504,13 @@ public: /// Returns a part in Active state with the given name or a part containing it. If there is no such part, returns nullptr. DataPartPtr getActiveContainingPart(const String & part_name) const; + DataPartPtr getActiveContainingPart(const String & part_name, DataPartsLock & lock) const; DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info) const; DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info, DataPartState state, DataPartsLock & lock) const; /// Swap part with it's identical copy (possible with another path on another disk). /// If original part is not active or doesn't exist exception will be thrown. - void swapActivePart(MergeTreeData::DataPartPtr part_copy); + void swapActivePart(MergeTreeData::DataPartPtr part_copy, DataPartsLock &); /// Returns all parts in specified partition DataPartsVector getVisibleDataPartsVectorInPartition(MergeTreeTransaction * txn, const String & partition_id, DataPartsLock * acquired_lock = nullptr) const; diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 8fa4ac6c78a..a8f34ba4cec 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -263,7 +263,10 @@ void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) cons if (moves_blocker.isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); - auto active_part = data->getActiveContainingPart(cloned_part.part->name); + /// `getActiveContainingPart` and `swapActivePart` are called under the same lock + /// to prevent part becoming inactive between calls + auto part_lock = data->lockParts(); + auto active_part = data->getActiveContainingPart(cloned_part.part->name, part_lock); /// It's ok, because we don't block moving parts for merges or mutations if (!active_part || active_part->name != cloned_part.part->name) @@ -284,7 +287,7 @@ void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) cons cloned_part.part->renameTo(active_part->name, false); /// TODO what happen if server goes down here? - data->swapActivePart(cloned_part.part); + data->swapActivePart(cloned_part.part, part_lock); LOG_TRACE(log, "Part {} was moved to {}", cloned_part.part->name, cloned_part.part->getDataPartStorage().getFullPath()); From 79b6792548c065d6795c5a167b29da44aa91dae7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Jun 2023 17:15:11 +0000 Subject: [PATCH 13/31] Enable allow_remove_stale_moving_parts for stateless tests --- tests/config/config.d/merge_tree_old_dirs_cleanup.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/merge_tree_old_dirs_cleanup.xml b/tests/config/config.d/merge_tree_old_dirs_cleanup.xml index 2b8ea63b63d..e6b50724c97 100644 --- a/tests/config/config.d/merge_tree_old_dirs_cleanup.xml +++ b/tests/config/config.d/merge_tree_old_dirs_cleanup.xml @@ -5,4 +5,5 @@ 5 + true From 53a30bee8d66c06f69f53b8850a11b8fbde2e1d4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 19:16:10 +0000 Subject: [PATCH 14/31] 00900_orc_load: kill less aggressively The test sporadically produces wrong results because the INSERT takes longer than 3 seconds, likely due to infrastructure latency. Removing the timeout to give it more headroom. Tests are afaik auto-killed after 10 min, so if there is a true issue in ClickHouse (e.g. deadlock), we would notice anyways. --- tests/queries/0_stateless/00900_orc_load.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00900_orc_load.sh b/tests/queries/0_stateless/00900_orc_load.sh index 62149fa554e..5dc5dfb87a7 100755 --- a/tests/queries/0_stateless/00900_orc_load.sh +++ b/tests/queries/0_stateless/00900_orc_load.sh @@ -12,6 +12,6 @@ ${CLICKHOUSE_CLIENT} --query="select * from orc_load FORMAT ORC" > "${CLICKHOUSE ${CLICKHOUSE_CLIENT} --query="truncate table orc_load" cat "${CLICKHOUSE_TMP}"/test.orc | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < "${CLICKHOUSE_TMP}"/test.orc +${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < "${CLICKHOUSE_TMP}"/test.orc ${CLICKHOUSE_CLIENT} --query="select * from orc_load" ${CLICKHOUSE_CLIENT} --query="drop table orc_load" From f12c257dc95de8188e2be8a70344b0bd0b1f6204 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Jun 2023 19:25:46 +0200 Subject: [PATCH 15/31] Fix --- src/Interpreters/Cache/FileCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index dc1f012f5c6..463d0d6605c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1017,7 +1017,8 @@ void FileCache::deactivateBackgroundOperations() metadata.cancelDownload(); for (auto & thread : download_threads) - thread.join(); + if (thread.joinable()) + thread.join(); } void FileCache::cleanup() From b0d4c9c83b210f9d266cc448df4641a07fdfb08c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 03:04:48 +0200 Subject: [PATCH 16/31] Disable hedged requests under TSan --- src/Core/SettingsQuirks.cpp | 9 ++- .../integration/test_hedged_requests/test.py | 75 +++++++++++++++++++ .../test_hedged_requests_parallel/test.py | 25 +++++++ tests/integration/test_secure_socket/test.py | 5 ++ ...851_hedged_connections_external_tables.sql | 1 + 5 files changed, 114 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 3326f42adf5..5b981c71403 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -1,10 +1,11 @@ +#include #include #include #include #include #include #include -#include + namespace { @@ -71,6 +72,12 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } } +#if defined(THREAD_SANITIZER) + settings.use_hedged_requests = false; + if (log) + LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); +#endif + if (!queryProfilerWorks()) { if (settings.query_profiler_real_time_period_ns) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index be6cea80f87..5de92f437c5 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -203,6 +203,11 @@ def update_configs( def test_stuck_replica(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs() cluster.pause_container("node_1") @@ -233,6 +238,11 @@ def test_stuck_replica(started_cluster): def test_long_query(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs() # Restart to reset pool states. @@ -249,12 +259,22 @@ def test_long_query(started_cluster): def test_send_table_status_sleep(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs(node_1_sleep_in_send_tables_status=sleep_time) check_query(expected_replica="node_2") check_changing_replica_events(1) def test_send_table_status_sleep2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=sleep_time, node_2_sleep_in_send_tables_status=sleep_time, @@ -264,12 +284,22 @@ def test_send_table_status_sleep2(started_cluster): def test_send_data(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs(node_1_sleep_in_send_data=sleep_time) check_query(expected_replica="node_2") check_changing_replica_events(1) def test_send_data2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time ) @@ -278,6 +308,11 @@ def test_send_data2(started_cluster): def test_combination1(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=sleep_time, node_2_sleep_in_send_data=sleep_time, @@ -287,6 +322,11 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_tables_status=sleep_time, @@ -296,6 +336,11 @@ def test_combination2(started_cluster): def test_combination3(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_tables_status=1000, @@ -306,6 +351,11 @@ def test_combination3(started_cluster): def test_combination4(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=1000, node_1_sleep_in_send_data=sleep_time, @@ -317,6 +367,11 @@ def test_combination4(started_cluster): def test_receive_timeout1(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + # Check the situation when first two replicas get receive timeout # in establishing connection, but the third replica is ok. update_configs( @@ -329,6 +384,11 @@ def test_receive_timeout1(started_cluster): def test_receive_timeout2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + # Check the situation when first replica get receive timeout # in packet receiving but there are replicas in process of # connection establishing. @@ -342,6 +402,11 @@ def test_receive_timeout2(started_cluster): def test_initial_receive_timeout(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + # Check the situation when replicas don't respond after # receiving query (so, no packets were send to initiator) update_configs( @@ -360,6 +425,11 @@ def test_initial_receive_timeout(started_cluster): def test_async_connect(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs() NODES["node"].restart_clickhouse() @@ -390,6 +460,11 @@ def test_async_connect(started_cluster): def test_async_query_sending(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_after_receiving_query=5000, node_2_sleep_after_receiving_query=5000, diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 492b869614f..34cad7bb754 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -172,6 +172,11 @@ def update_configs( def test_send_table_status_sleep(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=sleep_time, node_2_sleep_in_send_tables_status=sleep_time, @@ -181,6 +186,11 @@ def test_send_table_status_sleep(started_cluster): def test_send_data(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time ) @@ -189,6 +199,11 @@ def test_send_data(started_cluster): def test_combination1(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=1000, node_2_sleep_in_send_tables_status=1000, @@ -199,6 +214,11 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_tables_status=1000, @@ -210,6 +230,11 @@ def test_combination2(started_cluster): def test_query_with_no_data_to_sample(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time ) diff --git a/tests/integration/test_secure_socket/test.py b/tests/integration/test_secure_socket/test.py index 2dffbed03d6..827e4b81cfc 100644 --- a/tests/integration/test_secure_socket/test.py +++ b/tests/integration/test_secure_socket/test.py @@ -58,6 +58,11 @@ def test(started_cluster): config.format(sleep_in_send_data_ms=1000000), ) + if NODES["node1"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + attempts = 0 while attempts < 1000: setting = NODES["node2"].http_query( diff --git a/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql b/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql index c4625720e59..22888d5e68c 100644 --- a/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql +++ b/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql @@ -1 +1,2 @@ +-- Tags: no-tsan select number from remote('127.0.0.{3|2}', numbers(2)) where number global in (select number from numbers(1)) settings async_socket_for_remote=1, use_hedged_requests = 1, sleep_in_send_data_ms=10, receive_data_timeout_ms=1; From 6b7c17fb4c863b5145dca71daaf8c14cf4fa8a42 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 26 Jun 2023 01:24:09 +0000 Subject: [PATCH 17/31] Automatic style fix --- .../integration/test_hedged_requests/test.py | 62 +++++-------------- .../test_hedged_requests_parallel/test.py | 20 ++---- tests/integration/test_secure_socket/test.py | 4 +- 3 files changed, 22 insertions(+), 64 deletions(-) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 5de92f437c5..18ea3e50619 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -204,9 +204,7 @@ def update_configs( def test_stuck_replica(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs() @@ -239,9 +237,7 @@ def test_stuck_replica(started_cluster): def test_long_query(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs() @@ -260,9 +256,7 @@ def test_long_query(started_cluster): def test_send_table_status_sleep(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs(node_1_sleep_in_send_tables_status=sleep_time) check_query(expected_replica="node_2") @@ -271,9 +265,7 @@ def test_send_table_status_sleep(started_cluster): def test_send_table_status_sleep2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=sleep_time, @@ -285,9 +277,7 @@ def test_send_table_status_sleep2(started_cluster): def test_send_data(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs(node_1_sleep_in_send_data=sleep_time) check_query(expected_replica="node_2") @@ -296,9 +286,7 @@ def test_send_data(started_cluster): def test_send_data2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time @@ -309,9 +297,7 @@ def test_send_data2(started_cluster): def test_combination1(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=sleep_time, @@ -323,9 +309,7 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, @@ -337,9 +321,7 @@ def test_combination2(started_cluster): def test_combination3(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, @@ -352,9 +334,7 @@ def test_combination3(started_cluster): def test_combination4(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=1000, @@ -368,9 +348,7 @@ def test_combination4(started_cluster): def test_receive_timeout1(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") # Check the situation when first two replicas get receive timeout # in establishing connection, but the third replica is ok. @@ -385,9 +363,7 @@ def test_receive_timeout1(started_cluster): def test_receive_timeout2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") # Check the situation when first replica get receive timeout # in packet receiving but there are replicas in process of @@ -403,9 +379,7 @@ def test_receive_timeout2(started_cluster): def test_initial_receive_timeout(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") # Check the situation when replicas don't respond after # receiving query (so, no packets were send to initiator) @@ -426,9 +400,7 @@ def test_initial_receive_timeout(started_cluster): def test_async_connect(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs() @@ -461,10 +433,8 @@ def test_async_connect(started_cluster): def test_async_query_sending(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) - + pytest.skip("Hedged requests don't work under Thread Sanitizer") + update_configs( node_1_sleep_after_receiving_query=5000, node_2_sleep_after_receiving_query=5000, diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 34cad7bb754..728697c690d 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -173,9 +173,7 @@ def update_configs( def test_send_table_status_sleep(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=sleep_time, @@ -187,9 +185,7 @@ def test_send_table_status_sleep(started_cluster): def test_send_data(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time @@ -200,9 +196,7 @@ def test_send_data(started_cluster): def test_combination1(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=1000, @@ -215,9 +209,7 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, @@ -231,9 +223,7 @@ def test_combination2(started_cluster): def test_query_with_no_data_to_sample(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time diff --git a/tests/integration/test_secure_socket/test.py b/tests/integration/test_secure_socket/test.py index 827e4b81cfc..123715e5f05 100644 --- a/tests/integration/test_secure_socket/test.py +++ b/tests/integration/test_secure_socket/test.py @@ -59,9 +59,7 @@ def test(started_cluster): ) if NODES["node1"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") attempts = 0 while attempts < 1000: From a628bbb1f55bea23662ff65512b2e892310c13af Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:58:32 +0200 Subject: [PATCH 18/31] Fix tests --- src/Core/SettingsQuirks.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 5b981c71403..37a0f2db3e2 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -73,7 +73,7 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } #if defined(THREAD_SANITIZER) - settings.use_hedged_requests = false; + settings.use_hedged_requests.value = false; if (log) LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); #endif From 3b73e112165833e8baece650021adbb19e8f635f Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Jun 2023 11:04:40 +0000 Subject: [PATCH 19/31] Fix tests --- tests/queries/0_stateless/01361_fover_remote_num_tries.sh | 2 +- ...6_skip_unavailable_shards_excessive_attempts.reference | 8 ++++---- .../01956_skip_unavailable_shards_excessive_attempts.sh | 2 +- .../02226_parallel_reading_from_replicas_benchmark.sh | 2 ++ 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01361_fover_remote_num_tries.sh b/tests/queries/0_stateless/01361_fover_remote_num_tries.sh index 2ee2ec1bc76..f07ffc02e4f 100755 --- a/tests/queries/0_stateless/01361_fover_remote_num_tries.sh +++ b/tests/queries/0_stateless/01361_fover_remote_num_tries.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --connections_with_failover_max_tries 10 --query "SELECT hostName() FROM remote('128.1.2.3', default.tmp)" 2>&1 | grep -o -P 'Timeout exceeded while connecting to socket|Network is unreachable' | wc -l +$CLICKHOUSE_CLIENT --connections_with_failover_max_tries 10 --query "SELECT hostName() FROM remote('128.1.2.3', default.tmp)" 2>&1 | grep -o -P 'Timeout exceeded while connecting to socket|Network is unreachable|Timeout: connect timed out' | wc -l diff --git a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference index e39f4b962e6..ad409aac5b8 100644 --- a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference +++ b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference @@ -1,10 +1,10 @@ 255.255.255.255 -HedgedConnectionsFactory: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 executeQuery: Code: 519.: All attempts to get table structure failed. 127.2,255.255.255.255 0 -HedgedConnectionsFactory: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 255.255.255.255,127.2 0 -HedgedConnectionsFactory: Connection failed at try №1 -HedgedConnectionsFactory: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 diff --git a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh index 488e2fe106a..9f9de96ca6e 100755 --- a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh +++ b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh @@ -25,7 +25,7 @@ function execute_query() # clickhouse-client 2> >(wc -l) # # May dump output of "wc -l" after some other programs. - $CLICKHOUSE_CLIENT "${opts[@]}" --query "select * from remote('$hosts', system.one)" 2>"$stderr" + $CLICKHOUSE_CLIENT "${opts[@]}" --query "select * from remote('$hosts', system.one) settings use_hedged_requests=0" 2>"$stderr" process_log_safe "$stderr" } execute_query 255.255.255.255 diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index 4b8f8da5480..941f024825a 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -17,6 +17,8 @@ opts=( --allow_experimental_parallel_reading_from_replicas 1 --parallel_replicas_for_non_replicated_merge_tree 1 --max_parallel_replicas 3 + --use_hedged_requests 0 + --cluster_for_parallel_replicas parallel_replicas --iterations 1 ) From f134153f4e39befdbed703eda439d96e7d539a49 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 30 Jun 2023 13:12:04 +0200 Subject: [PATCH 20/31] Fix race with "user_was_dropped" in ContextAccess. --- src/Access/ContextAccess.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 63604a03b4e..1259d8d72eb 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -221,7 +221,7 @@ private: mutable Poco::Logger * trace_log = nullptr; mutable UserPtr user; mutable String user_name; - mutable bool user_was_dropped = false; + mutable std::atomic user_was_dropped = false; mutable scope_guard subscription_for_user_change; mutable std::shared_ptr enabled_roles; mutable scope_guard subscription_for_roles_changes; From 2311dd9b5ddbeb8587ce7201b889c0ab1442d003 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 30 Jun 2023 15:18:54 +0200 Subject: [PATCH 21/31] Use TSA_GUARDED_BY() macro in the definition of ContextAccess. --- src/Access/ContextAccess.cpp | 16 +++++++------- src/Access/ContextAccess.h | 43 ++++++++++++++++++++++-------------- 2 files changed, 34 insertions(+), 25 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 0d28edc0a10..9c57853679f 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -228,6 +228,12 @@ ContextAccess::ContextAccess(const AccessControl & access_control_, const Params } +ContextAccess::ContextAccess(FullAccess) + : is_full_access(true), access(std::make_shared(AccessRights::getFullAccess())), access_with_implicit(access) +{ +} + + ContextAccess::~ContextAccess() { enabled_settings.reset(); @@ -413,14 +419,8 @@ std::optional ContextAccess::getQuotaUsage() const std::shared_ptr ContextAccess::getFullAccess() { - static const std::shared_ptr res = [] - { - auto full_access = std::make_shared(); - full_access->is_full_access = true; - full_access->access = std::make_shared(AccessRights::getFullAccess()); - full_access->access_with_implicit = full_access->access; - return full_access; - }(); + static const std::shared_ptr res = + [] { return std::shared_ptr(new ContextAccess{kFullAccess}); }(); return res; } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 1259d8d72eb..60bad0118fc 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -69,7 +69,6 @@ public: using Params = ContextAccessParams; const Params & getParams() const { return params; } - ContextAccess() { } /// NOLINT ContextAccess(const AccessControl & access_control_, const Params & params_); /// Returns the current user. Throws if user is nullptr. @@ -171,10 +170,17 @@ public: private: friend class AccessControl; + struct FullAccess {}; + static const FullAccess kFullAccess; + + /// Makes an instance of ContextAccess which provides full access to everything + /// without any limitations. This is used for the global context. + explicit ContextAccess(FullAccess); + void initialize(); - void setUser(const UserPtr & user_) const; - void setRolesInfo(const std::shared_ptr & roles_info_) const; - void calculateAccessRights() const; + void setUser(const UserPtr & user_) const TSA_REQUIRES(mutex); + void setRolesInfo(const std::shared_ptr & roles_info_) const TSA_REQUIRES(mutex); + void calculateAccessRights() const TSA_REQUIRES(mutex); template bool checkAccessImpl(const AccessFlags & flags) const; @@ -217,20 +223,23 @@ private: const AccessControl * access_control = nullptr; const Params params; - bool is_full_access = false; - mutable Poco::Logger * trace_log = nullptr; - mutable UserPtr user; - mutable String user_name; + const bool is_full_access = false; + mutable std::atomic user_was_dropped = false; - mutable scope_guard subscription_for_user_change; - mutable std::shared_ptr enabled_roles; - mutable scope_guard subscription_for_roles_changes; - mutable std::shared_ptr roles_info; - mutable std::shared_ptr access; - mutable std::shared_ptr access_with_implicit; - mutable std::shared_ptr enabled_row_policies; - mutable std::shared_ptr enabled_quota; - mutable std::shared_ptr enabled_settings; + mutable std::atomic trace_log = nullptr; + + mutable UserPtr user TSA_GUARDED_BY(mutex); + mutable String user_name TSA_GUARDED_BY(mutex); + mutable scope_guard subscription_for_user_change TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_roles TSA_GUARDED_BY(mutex); + mutable scope_guard subscription_for_roles_changes TSA_GUARDED_BY(mutex); + mutable std::shared_ptr roles_info TSA_GUARDED_BY(mutex); + mutable std::shared_ptr access TSA_GUARDED_BY(mutex); + mutable std::shared_ptr access_with_implicit TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_row_policies TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_quota TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_settings TSA_GUARDED_BY(mutex); + mutable std::mutex mutex; }; From 3f29bd1bd42f7ebff737d2196c285aa7be4c6989 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Jun 2023 18:20:59 +0200 Subject: [PATCH 22/31] Try fix test --- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index b9f9548a7b3..509d1a635b1 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-random-settings +# Tags: no-fasttest, no-parallel, no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 292eec247058c6020c48d0d4c3ebd27784fed466 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 3 Jul 2023 09:40:36 +0200 Subject: [PATCH 23/31] Run cargo update to fix build with nightly --- rust/skim/Cargo.lock | 360 ++++++++++++++++++++++++++----------------- 1 file changed, 218 insertions(+), 142 deletions(-) diff --git a/rust/skim/Cargo.lock b/rust/skim/Cargo.lock index badd22dad07..9f948ee1c38 100644 --- a/rust/skim/Cargo.lock +++ b/rust/skim/Cargo.lock @@ -14,13 +14,19 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "0.7.20" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" +checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" dependencies = [ "memchr", ] +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + [[package]] name = "android_system_properties" version = "0.1.5" @@ -32,9 +38,9 @@ dependencies = [ [[package]] name = "arrayvec" -version = "0.7.2" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "atty" @@ -42,7 +48,7 @@ version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" dependencies = [ - "hermit-abi", + "hermit-abi 0.1.19", "libc", "winapi", ] @@ -67,15 +73,15 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bumpalo" -version = "3.11.1" +version = "3.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "572f695136211188308f16ad2ca5c851a712c464060ae6974944458eb83880ba" +checksum = "a3e2c3daef883ecc1b5d58c15adae93470a91d425f3532ba1695849656af3fc1" [[package]] name = "cc" -version = "1.0.77" +version = "1.0.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9f73505338f7d905b19d18738976aae232eb46b8efc15554ffc56deb5d9ebe4" +checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" [[package]] name = "cfg-if" @@ -85,13 +91,13 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.23" +version = "0.4.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16b0a3d9ed01224b22057780a37bb8c5dbfe1be8ba48678e7bf57ec4b385411f" +checksum = "ec837a71355b28f6556dbd569b37b3f363091c0bd4b2e735674521b4c5fd9bc5" dependencies = [ + "android-tzdata", "iana-time-zone", "js-sys", - "num-integer", "num-traits", "time 0.1.45", "wasm-bindgen", @@ -100,9 +106,9 @@ dependencies = [ [[package]] name = "clap" -version = "3.2.23" +version = "3.2.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71655c45cb9845d3270c9d6df84ebe72b4dad3c2ba3f7023ad47c144e4e473a5" +checksum = "4ea181bf566f71cb9a5d17a59e1871af638180a18fb0035c92ae62b705207123" dependencies = [ "atty", "bitflags", @@ -135,9 +141,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" +checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" [[package]] name = "crossbeam" @@ -155,9 +161,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.6" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" +checksum = "a33c2bf77f2df06183c3aa30d1e96c0695a313d4f9c453cc3762a6db39f99200" dependencies = [ "cfg-if", "crossbeam-utils", @@ -165,9 +171,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" +checksum = "ce6fd6f855243022dcecf8702fef0c297d4338e226845fe067f6341ad9fa0cef" dependencies = [ "cfg-if", "crossbeam-epoch", @@ -176,14 +182,14 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.13" +version = "0.9.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01a9af1f4c2ef74bb8aa1f7e19706bc72d03598c8a570bb5de72243c7a9d9d5a" +checksum = "ae211234986c545741a7dc064309f67ee1e5ad243d0e48335adc0484d960bcc7" dependencies = [ "autocfg", "cfg-if", "crossbeam-utils", - "memoffset 0.7.1", + "memoffset 0.9.0", "scopeguard", ] @@ -199,18 +205,18 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.14" +version = "0.8.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fb766fa798726286dbbb842f174001dab8abc7b627a1dd86e0b7222a95d929f" +checksum = "5a22b2d63d4d1dc0b7f1b6b2747dd0088008a9be28b6ddf0b1e7d335e3037294" dependencies = [ "cfg-if", ] [[package]] name = "cxx" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdf07d07d6531bfcdbe9b8b739b104610c6508dcc4d63b410585faf338241daf" +checksum = "e88abab2f5abbe4c56e8f1fb431b784d710b709888f35755a160e62e33fe38e8" dependencies = [ "cc", "cxxbridge-flags", @@ -220,9 +226,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2eb5b96ecdc99f72657332953d4d9c50135af1bac34277801cc3937906ebd39" +checksum = "5c0c11acd0e63bae27dcd2afced407063312771212b7a823b4fd72d633be30fb" dependencies = [ "cc", "codespan-reporting", @@ -230,31 +236,31 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn", + "syn 2.0.23", ] [[package]] name = "cxxbridge-flags" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac040a39517fd1674e0f32177648334b0f4074625b5588a64519804ba0553b12" +checksum = "8d3816ed957c008ccd4728485511e3d9aaf7db419aa321e3d2c5a2f3411e36c8" [[package]] name = "cxxbridge-macro" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1362b0ddcfc4eb0a1f57b68bd77dd99f0e826958a96abd0ae9bd092e114ffed6" +checksum = "a26acccf6f445af85ea056362561a24ef56cdc15fcc685f03aec50b9c702cb6d" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.23", ] [[package]] name = "darling" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0dd3cd20dc6b5a876612a6e5accfe7f3dd883db6d07acfbf14c128f61550dfa" +checksum = "7b750cb3417fd1b327431a470f388520309479ab0bf5e323505daf0290cd3850" dependencies = [ "darling_core", "darling_macro", @@ -262,27 +268,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a784d2ccaf7c98501746bf0be29b2022ba41fd62a2e622af997a03e9f972859f" +checksum = "109c1ca6e6b7f82cc233a97004ea8ed7ca123a9af07a8230878fcfda9b158bf0" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim", - "syn", + "syn 1.0.109", ] [[package]] name = "darling_macro" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7618812407e9402654622dd402b0a89dff9ba93badd6540781526117b92aab7e" +checksum = "a4aab4dbc9f7611d8b55048a3a16d2d010c2c8334e46304b40ac1cc14bf3b48e" dependencies = [ "darling_core", "quote", - "syn", + "syn 1.0.109", ] [[package]] @@ -313,7 +319,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn", + "syn 1.0.109", ] [[package]] @@ -323,7 +329,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f0314b72bed045f3a68671b3c86328386762c93f82d98c65c3cb5e5f573dd68" dependencies = [ "derive_builder_core", - "syn", + "syn 1.0.109", ] [[package]] @@ -349,9 +355,9 @@ dependencies = [ [[package]] name = "either" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90e5c1c8368803113bf0c9584fc495a58b86dc8a29edbf8fe877d21d9507e797" +checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" [[package]] name = "env_logger" @@ -383,9 +389,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.8" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c05aeb6a22b8f62540c194aac980f2115af067bfe15a0734d7277a768d396b31" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" dependencies = [ "cfg-if", "libc", @@ -407,6 +413,12 @@ dependencies = [ "libc", ] +[[package]] +name = "hermit-abi" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" + [[package]] name = "humantime" version = "2.1.0" @@ -415,26 +427,25 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "iana-time-zone" -version = "0.1.53" +version = "0.1.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64c122667b287044802d6ce17ee2ddf13207ed924c712de9a66a5814d5b64765" +checksum = "2fad5b825842d2b38bd206f3e81d6957625fd7f0a361e345c30e01a0ae2dd613" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "winapi", + "windows", ] [[package]] name = "iana-time-zone-haiku" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" dependencies = [ - "cxx", - "cxx-build", + "cc", ] [[package]] @@ -445,9 +456,9 @@ checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] name = "indexmap" -version = "1.9.2" +version = "1.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885e79c1fc4b10f0e172c475f458b7f7b93061064d98c3293e98c5ba0c8b399" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" dependencies = [ "autocfg", "hashbrown", @@ -455,9 +466,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.60" +version = "0.3.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49409df3e3bf0856b916e2ceaca09ee28e6871cf7d9ce97a692cacfdb2a25a47" +checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" dependencies = [ "wasm-bindgen", ] @@ -470,27 +481,24 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.138" +version = "0.2.147" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db6d7e329c562c5dfab7a46a2afabc8b987ab9a4834c9d1ca04dc54c1546cef8" +checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" [[package]] name = "link-cplusplus" -version = "1.0.7" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9272ab7b96c9046fbc5bc56c06c117cb639fe2d509df0c421cad82d2915cf369" +checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" dependencies = [ "cc", ] [[package]] name = "log" -version = "0.4.17" +version = "0.4.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" -dependencies = [ - "cfg-if", -] +checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" [[package]] name = "memchr" @@ -509,9 +517,9 @@ dependencies = [ [[package]] name = "memoffset" -version = "0.7.1" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5de893c32cde5f383baa4c04c5d6dbdd735cfd4a794b0debdb2bb1b421da5ff4" +checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" dependencies = [ "autocfg", ] @@ -541,16 +549,6 @@ dependencies = [ "pin-utils", ] -[[package]] -name = "num-integer" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" -dependencies = [ - "autocfg", - "num-traits", -] - [[package]] name = "num-traits" version = "0.2.15" @@ -562,25 +560,25 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.14.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6058e64324c71e02bc2b150e4f3bc8286db6c83092132ffa3f6b1eab0f9def5" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi", + "hermit-abi 0.3.1", "libc", ] [[package]] name = "once_cell" -version = "1.16.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86f0b0d4bf799edbc74508c1e8bf170ff5f41238e5f8225603ca7caaae2b7860" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" [[package]] name = "os_str_bytes" -version = "6.4.1" +version = "6.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7820b9daea5457c9f21c69448905d723fbd21136ccf521748f23fd49e723ee" +checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" [[package]] name = "pin-utils" @@ -590,27 +588,27 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "proc-macro2" -version = "1.0.47" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ea3d908b0e36316caf9e9e2c4625cdde190a7e6f440d794667ed17a1855e725" +checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" dependencies = [ "unicode-ident", ] [[package]] name = "quote" -version = "1.0.21" +version = "1.0.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbe448f377a7d6961e30f5955f9b8d106c3f5e449d493ee1b125c1d43c2b5179" +checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" dependencies = [ "proc-macro2", ] [[package]] name = "rayon" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6db3a213adf02b3bcfd2d3846bb41cb22857d131789e01df434fb7e7bc0759b7" +checksum = "1d2df5196e37bcc87abebc0053e20787d73847bb33134a69841207dd0a47f03b" dependencies = [ "either", "rayon-core", @@ -618,9 +616,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.10.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cac410af5d00ab6884528b4ab69d1e8e146e8d471201800fa1b4524126de6ad3" +checksum = "4b8f95bd6966f5c87776639160a66bd8ab9895d9d4ab01ddba9fc60661aebe8d" dependencies = [ "crossbeam-channel", "crossbeam-deque", @@ -650,9 +648,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.7.0" +version = "1.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e076559ef8e241f2ae3479e36f97bd5741c0330689e217ad51ce2c76808b868a" +checksum = "d0ab3ca65655bb1e41f2a8c8cd662eb4fb035e67c3f78da1d61dffe89d07300f" dependencies = [ "aho-corasick", "memchr", @@ -661,15 +659,15 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.6.28" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "456c603be3e8d448b072f410900c09faf164fbce2d480456f50eea6e25f9c848" +checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" [[package]] name = "rustversion" -version = "1.0.9" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97477e48b4cf8603ad5f7aaf897467cf42ab4218a38ef76fb14c2d6773a6d6a8" +checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" [[package]] name = "scopeguard" @@ -679,15 +677,15 @@ checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" [[package]] name = "scratch" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8132065adcfd6e02db789d9285a0deb2f3fcb04002865ab67d5fb103533898" +checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" [[package]] name = "serde" -version = "1.0.149" +version = "1.0.164" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "256b9932320c590e707b94576e3cc1f7c9024d0ee6612dfbcf1cb106cbe8e055" +checksum = "9e8c8cf938e98f769bc164923b06dce91cea1751522f46f8466461af04c9027d" [[package]] name = "shlex" @@ -697,9 +695,9 @@ checksum = "43b2853a4d09f215c24cc5489c992ce46052d359b5109343cbafbf26bc62f8a3" [[package]] name = "skim" -version = "0.10.2" +version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cebed5f897cd6c0d80fbe30adb36c0abf7400e93043a63ae56458495642b3485" +checksum = "e5d28de0a6cb2cdd83a076f1de9d965b973ae08b244df1aa70b432946dda0f32" dependencies = [ "atty", "beef", @@ -717,7 +715,7 @@ dependencies = [ "rayon", "regex", "shlex", - "time 0.3.17", + "time 0.3.22", "timer", "tuikit", "unicode-width", @@ -732,9 +730,20 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "syn" -version = "1.0.105" +version = "1.0.109" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60b9b43d45702de4c839cb9b51d9f529c5dd26a4aff255b42b1ebc03e88ee908" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" dependencies = [ "proc-macro2", "quote", @@ -754,9 +763,9 @@ dependencies = [ [[package]] name = "termcolor" -version = "1.1.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bab24d30b911b2376f3a13cc2cd443142f0c81dda04c118693e35b3835757755" +checksum = "be55cf8942feac5c765c2c993422806843c9a9a45d4d5c407ad6dd2ea95eb9b6" dependencies = [ "winapi-util", ] @@ -769,30 +778,31 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.37" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10deb33631e3c9018b9baf9dcbbc4f737320d2b576bac10f6aefa048fa407e3e" +checksum = "978c9a314bd8dc99be594bc3c175faaa9794be04a5a5e153caba6915336cebac" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.37" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "982d17546b47146b28f7c22e3d08465f6b8903d0ea13c1660d9d84a6e7adcdbb" +checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.23", ] [[package]] name = "thread_local" -version = "1.1.4" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5516c27b78311c50bf42c071425c560ac799b11c30b31f87e3081965fe5e0180" +checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" dependencies = [ + "cfg-if", "once_cell", ] @@ -809,9 +819,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.17" +version = "0.3.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a561bf4617eebd33bca6434b988f39ed798e527f51a1e797d0ee4f61c0a38376" +checksum = "ea9e1b3cf1243ae005d9e74085d4d542f3125458f3a81af210d901dcd7411efd" dependencies = [ "serde", "time-core", @@ -819,9 +829,9 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" +checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" [[package]] name = "timer" @@ -848,9 +858,9 @@ dependencies = [ [[package]] name = "unicode-ident" -version = "1.0.5" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ceab39d59e4c9499d4e5a8ee0e2735b891bb7308ac83dfb4e80cad195c9f6f3" +checksum = "b15811caf2415fb889178633e7724bad2509101cde276048e013b9def5e51fa0" [[package]] name = "unicode-width" @@ -860,15 +870,15 @@ checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" [[package]] name = "utf8parse" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "936e4b492acfd135421d8dca4b1aa80a7bfc26e702ef3af710e0752684df5372" +checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "vte" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1aae21c12ad2ec2d168c236f369c38ff332bc1134f7246350dca641437365045" +checksum = "f5022b5fbf9407086c180e9557be968742d839e68346af7792b8592489732197" dependencies = [ "arrayvec", "utf8parse", @@ -899,9 +909,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eaf9f5aceeec8be17c128b2e93e031fb8a4d469bb9c4ae2d7dc1888b26887268" +checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -909,24 +919,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c8ffb332579b0557b52d268b91feab8df3615f265d5270fec2a8c95b17c1142" +checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn", + "syn 2.0.23", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "052be0f94026e6cbc75cdefc9bae13fd6052cdcaf532fa6c45e7ae33a1e6c810" +checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -934,22 +944,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07bc0c051dc5f23e307b13285f9d75df86bfdf816c5721e573dec1f9b8aa193c" +checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.23", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c38c045535d93ec4f0b4defec448e4291638ee608530863b1e2ba115d4fff7f" +checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" [[package]] name = "winapi" @@ -981,3 +991,69 @@ name = "winapi-x86_64-pc-windows-gnu" version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e686886bc078bc1b0b600cac0147aadb815089b6e4da64016cbd754b6342700f" +dependencies = [ + "windows-targets", +] + +[[package]] +name = "windows-targets" +version = "0.48.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05d4b17490f70499f20b9e791dcf6a299785ce8af4d709018206dc5b4953e95f" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91ae572e1b79dba883e0d315474df7305d12f569b400fcf90581b06062f7e1bc" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2ef27e0d7bdfcfc7b868b317c1d32c641a6fe4629c171b8928c7b08d98d7cf3" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "622a1962a7db830d6fd0a69683c80a18fda201879f0f447f065a3b7467daa241" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4542c6e364ce21bf45d69fdd2a8e455fa38d316158cfd43b3ac1c5b1b19f8e00" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca2b8a661f7628cbd23440e50b05d705db3686f894fc9580820623656af974b1" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7896dbc1f41e08872e9d5e8f8baa8fdd2677f29468c4e156210174edc7f7b953" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" From 579a446d1901b356fd1f58d98cd1cf6898de2604 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 3 Jul 2023 13:49:56 +0200 Subject: [PATCH 24/31] Update 02789_filesystem_cache_alignment.sh --- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index 509d1a635b1..912cdd3d1e8 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -61,7 +61,7 @@ WHERE file_segment_size < file_size AND end_offset + 1 != file_size; ") #echo $all -if [ "$all" -gt "10" ]; then +if [ "$all" -gt "1" ]; then echo "OK" else echo "FAIL" @@ -105,7 +105,7 @@ SELECT count() FROM ($query2) WHERE file_segment_size < file_size AND file_segment_range_end + 1 != file_size; ") -if [ "$all" -gt "10" ]; then +if [ "$all" -gt "1" ]; then echo "OK" else echo "FAIL" From d1c4a37473749172406e850a3f0c0bcc6f122a49 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 3 Jul 2023 14:38:38 +0200 Subject: [PATCH 25/31] Remove MemoryPool from Poco because it's useless --- base/poco/Foundation/CMakeLists.txt | 1 - .../poco/Foundation/include/Poco/MemoryPool.h | 116 ------------------ base/poco/Foundation/src/MemoryPool.cpp | 105 ---------------- .../Net/include/Poco/Net/HTTPBasicStreamBuf.h | 5 +- .../include/Poco/Net/HTTPBufferAllocator.h | 53 -------- .../Net/include/Poco/Net/HTTPChunkedStream.h | 13 -- .../include/Poco/Net/HTTPFixedLengthStream.h | 12 -- .../Net/include/Poco/Net/HTTPHeaderStream.h | 13 -- base/poco/Net/include/Poco/Net/HTTPStream.h | 13 -- base/poco/Net/src/HTTPBufferAllocator.cpp | 44 ------- base/poco/Net/src/HTTPChunkedStream.cpp | 50 +------- base/poco/Net/src/HTTPFixedLengthStream.cpp | 47 +------ base/poco/Net/src/HTTPHeaderStream.cpp | 50 +------- base/poco/Net/src/HTTPSession.cpp | 8 +- base/poco/Net/src/HTTPStream.cpp | 48 +------- 15 files changed, 10 insertions(+), 568 deletions(-) delete mode 100644 base/poco/Foundation/include/Poco/MemoryPool.h delete mode 100644 base/poco/Foundation/src/MemoryPool.cpp delete mode 100644 base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h delete mode 100644 base/poco/Net/src/HTTPBufferAllocator.cpp diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 358f49ed055..d0dde8a51a5 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -87,7 +87,6 @@ set (SRCS src/LoggingRegistry.cpp src/LogStream.cpp src/MD5Engine.cpp - src/MemoryPool.cpp src/MemoryStream.cpp src/Message.cpp src/Mutex.cpp diff --git a/base/poco/Foundation/include/Poco/MemoryPool.h b/base/poco/Foundation/include/Poco/MemoryPool.h deleted file mode 100644 index 9ab12081b5f..00000000000 --- a/base/poco/Foundation/include/Poco/MemoryPool.h +++ /dev/null @@ -1,116 +0,0 @@ -// -// MemoryPool.h -// -// Library: Foundation -// Package: Core -// Module: MemoryPool -// -// Definition of the MemoryPool class. -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef Foundation_MemoryPool_INCLUDED -#define Foundation_MemoryPool_INCLUDED - - -#include -#include -#include "Poco/Foundation.h" -#include "Poco/Mutex.h" - - -namespace Poco -{ - - -class Foundation_API MemoryPool -/// A simple pool for fixed-size memory blocks. -/// -/// The main purpose of this class is to speed-up -/// memory allocations, as well as to reduce memory -/// fragmentation in situations where the same blocks -/// are allocated all over again, such as in server -/// applications. -/// -/// All allocated blocks are retained for future use. -/// A limit on the number of blocks can be specified. -/// Blocks can be preallocated. -{ -public: - MemoryPool(std::size_t blockSize, int preAlloc = 0, int maxAlloc = 0); - /// Creates a MemoryPool for blocks with the given blockSize. - /// The number of blocks given in preAlloc are preallocated. - - ~MemoryPool(); - - void * get(); - /// Returns a memory block. If there are no more blocks - /// in the pool, a new block will be allocated. - /// - /// If maxAlloc blocks are already allocated, an - /// OutOfMemoryException is thrown. - - void release(void * ptr); - /// Releases a memory block and returns it to the pool. - - std::size_t blockSize() const; - /// Returns the block size. - - int allocated() const; - /// Returns the number of allocated blocks. - - int available() const; - /// Returns the number of available blocks in the pool. - -private: - MemoryPool(); - MemoryPool(const MemoryPool &); - MemoryPool & operator=(const MemoryPool &); - - void clear(); - - enum - { - BLOCK_RESERVE = 128 - }; - - typedef std::vector BlockVec; - - std::size_t _blockSize; - int _maxAlloc; - int _allocated; - BlockVec _blocks; - FastMutex _mutex; -}; - - -// -// inlines -// -inline std::size_t MemoryPool::blockSize() const -{ - return _blockSize; -} - - -inline int MemoryPool::allocated() const -{ - return _allocated; -} - - -inline int MemoryPool::available() const -{ - return (int)_blocks.size(); -} - - -} // namespace Poco - - -#endif // Foundation_MemoryPool_INCLUDED diff --git a/base/poco/Foundation/src/MemoryPool.cpp b/base/poco/Foundation/src/MemoryPool.cpp deleted file mode 100644 index 01c477be525..00000000000 --- a/base/poco/Foundation/src/MemoryPool.cpp +++ /dev/null @@ -1,105 +0,0 @@ -// -// MemoryPool.cpp -// -// Library: Foundation -// Package: Core -// Module: MemoryPool -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MemoryPool.h" -#include "Poco/Exception.h" - - -namespace Poco { - - -MemoryPool::MemoryPool(std::size_t blockSize, int preAlloc, int maxAlloc): - _blockSize(blockSize), - _maxAlloc(maxAlloc), - _allocated(preAlloc) -{ - poco_assert (maxAlloc == 0 || maxAlloc >= preAlloc); - poco_assert (preAlloc >= 0 && maxAlloc >= 0); - - int r = BLOCK_RESERVE; - if (preAlloc > r) - r = preAlloc; - if (maxAlloc > 0 && maxAlloc < r) - r = maxAlloc; - _blocks.reserve(r); - - try - { - for (int i = 0; i < preAlloc; ++i) - { - _blocks.push_back(new char[_blockSize]); - } - } - catch (...) - { - clear(); - throw; - } -} - - -MemoryPool::~MemoryPool() -{ - clear(); -} - - -void MemoryPool::clear() -{ - for (BlockVec::iterator it = _blocks.begin(); it != _blocks.end(); ++it) - { - delete [] *it; - } - _blocks.clear(); -} - - -void* MemoryPool::get() -{ - FastMutex::ScopedLock lock(_mutex); - - if (_blocks.empty()) - { - if (_maxAlloc == 0 || _allocated < _maxAlloc) - { - ++_allocated; - return new char[_blockSize]; - } - else throw OutOfMemoryException("MemoryPool exhausted"); - } - else - { - char* ptr = _blocks.back(); - _blocks.pop_back(); - return ptr; - } -} - - -void MemoryPool::release(void* ptr) -{ - FastMutex::ScopedLock lock(_mutex); - - try - { - _blocks.push_back(reinterpret_cast(ptr)); - } - catch (...) - { - delete [] reinterpret_cast(ptr); - } -} - - -} // namespace Poco diff --git a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h index c4872d95353..c87719b63a4 100644 --- a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h +++ b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h @@ -19,7 +19,6 @@ #include "Poco/BufferedStreamBuf.h" -#include "Poco/Net/HTTPBufferAllocator.h" #include "Poco/Net/Net.h" @@ -27,9 +26,9 @@ namespace Poco { namespace Net { + constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 8 * 1024; - - typedef Poco::BasicBufferedStreamBuf, HTTPBufferAllocator> HTTPBasicStreamBuf; + typedef Poco::BasicBufferedStreamBuf> HTTPBasicStreamBuf; } diff --git a/base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h b/base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h deleted file mode 100644 index 5d088e35297..00000000000 --- a/base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h +++ /dev/null @@ -1,53 +0,0 @@ -// -// HTTPBufferAllocator.h -// -// Library: Net -// Package: HTTP -// Module: HTTPBufferAllocator -// -// Definition of the HTTPBufferAllocator class. -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef Net_HTTPBufferAllocator_INCLUDED -#define Net_HTTPBufferAllocator_INCLUDED - - -#include -#include "Poco/MemoryPool.h" -#include "Poco/Net/Net.h" - - -namespace Poco -{ -namespace Net -{ - - - class Net_API HTTPBufferAllocator - /// A BufferAllocator for HTTP streams. - { - public: - static char * allocate(std::streamsize size); - static void deallocate(char * ptr, std::streamsize size); - - enum - { - BUFFER_SIZE = 128 * 1024 - }; - - private: - static Poco::MemoryPool _pool; - }; - - -} -} // namespace Poco::Net - - -#endif // Net_HTTPBufferAllocator_INCLUDED diff --git a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h index 47987b18817..5f4729c9278 100644 --- a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h @@ -21,7 +21,6 @@ #include #include #include -#include "Poco/MemoryPool.h" #include "Poco/Net/HTTPBasicStreamBuf.h" #include "Poco/Net/Net.h" @@ -80,12 +79,6 @@ namespace Net public: HTTPChunkedInputStream(HTTPSession & session); ~HTTPChunkedInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -95,12 +88,6 @@ namespace Net public: HTTPChunkedOutputStream(HTTPSession & session); ~HTTPChunkedOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h index 4de211fdb92..2f4df102605 100644 --- a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h @@ -78,12 +78,6 @@ namespace Net public: HTTPFixedLengthInputStream(HTTPSession & session, HTTPFixedLengthStreamBuf::ContentLength length); ~HTTPFixedLengthInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -93,12 +87,6 @@ namespace Net public: HTTPFixedLengthOutputStream(HTTPSession & session, HTTPFixedLengthStreamBuf::ContentLength length); ~HTTPFixedLengthOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h b/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h index bcfca984d8b..cf1a6dba2e6 100644 --- a/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h @@ -21,7 +21,6 @@ #include #include #include -#include "Poco/MemoryPool.h" #include "Poco/Net/HTTPBasicStreamBuf.h" #include "Poco/Net/Net.h" @@ -74,12 +73,6 @@ namespace Net public: HTTPHeaderInputStream(HTTPSession & session); ~HTTPHeaderInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -89,12 +82,6 @@ namespace Net public: HTTPHeaderOutputStream(HTTPSession & session); ~HTTPHeaderOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPStream.h b/base/poco/Net/include/Poco/Net/HTTPStream.h index 0197bc62eb2..48502347b2c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPStream.h @@ -21,7 +21,6 @@ #include #include #include -#include "Poco/MemoryPool.h" #include "Poco/Net/HTTPBasicStreamBuf.h" #include "Poco/Net/Net.h" @@ -75,12 +74,6 @@ namespace Net public: HTTPInputStream(HTTPSession & session); ~HTTPInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -90,12 +83,6 @@ namespace Net public: HTTPOutputStream(HTTPSession & session); ~HTTPOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/src/HTTPBufferAllocator.cpp b/base/poco/Net/src/HTTPBufferAllocator.cpp deleted file mode 100644 index 2944e2a6121..00000000000 --- a/base/poco/Net/src/HTTPBufferAllocator.cpp +++ /dev/null @@ -1,44 +0,0 @@ -// -// HTTPBufferAllocator.cpp -// -// Library: Net -// Package: HTTP -// Module: HTTPBufferAllocator -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/Net/HTTPBufferAllocator.h" - - -using Poco::MemoryPool; - - -namespace Poco { -namespace Net { - - -MemoryPool HTTPBufferAllocator::_pool(HTTPBufferAllocator::BUFFER_SIZE, 16); - - -char* HTTPBufferAllocator::allocate(std::streamsize size) -{ - poco_assert_dbg (size == BUFFER_SIZE); - - return reinterpret_cast(_pool.get()); -} - - -void HTTPBufferAllocator::deallocate(char* ptr, std::streamsize size) -{ - poco_assert_dbg (size == BUFFER_SIZE); - - _pool.release(ptr); -} - - -} } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPChunkedStream.cpp b/base/poco/Net/src/HTTPChunkedStream.cpp index f2f79da590b..376e3f55492 100644 --- a/base/poco/Net/src/HTTPChunkedStream.cpp +++ b/base/poco/Net/src/HTTPChunkedStream.cpp @@ -34,7 +34,7 @@ namespace Net { HTTPChunkedStreamBuf::HTTPChunkedStreamBuf(HTTPSession& session, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _mode(mode), _chunk(0) @@ -181,10 +181,6 @@ HTTPChunkedStreamBuf* HTTPChunkedIOS::rdbuf() // HTTPChunkedInputStream // - -Poco::MemoryPool HTTPChunkedInputStream::_pool(sizeof(HTTPChunkedInputStream)); - - HTTPChunkedInputStream::HTTPChunkedInputStream(HTTPSession& session): HTTPChunkedIOS(session, std::ios::in), std::istream(&_buf) @@ -196,34 +192,10 @@ HTTPChunkedInputStream::~HTTPChunkedInputStream() { } - -void* HTTPChunkedInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPChunkedInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPChunkedOutputStream // - -Poco::MemoryPool HTTPChunkedOutputStream::_pool(sizeof(HTTPChunkedOutputStream)); - - HTTPChunkedOutputStream::HTTPChunkedOutputStream(HTTPSession& session): HTTPChunkedIOS(session, std::ios::out), std::ostream(&_buf) @@ -235,24 +207,4 @@ HTTPChunkedOutputStream::~HTTPChunkedOutputStream() { } - -void* HTTPChunkedOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPChunkedOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPFixedLengthStream.cpp b/base/poco/Net/src/HTTPFixedLengthStream.cpp index d19f6122ee1..fd77ff71cd9 100644 --- a/base/poco/Net/src/HTTPFixedLengthStream.cpp +++ b/base/poco/Net/src/HTTPFixedLengthStream.cpp @@ -30,7 +30,7 @@ namespace Net { HTTPFixedLengthStreamBuf::HTTPFixedLengthStreamBuf(HTTPSession& session, ContentLength length, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _length(length), _count(0) @@ -109,9 +109,6 @@ HTTPFixedLengthStreamBuf* HTTPFixedLengthIOS::rdbuf() // -Poco::MemoryPool HTTPFixedLengthInputStream::_pool(sizeof(HTTPFixedLengthInputStream)); - - HTTPFixedLengthInputStream::HTTPFixedLengthInputStream(HTTPSession& session, HTTPFixedLengthStreamBuf::ContentLength length): HTTPFixedLengthIOS(session, length, std::ios::in), std::istream(&_buf) @@ -124,33 +121,10 @@ HTTPFixedLengthInputStream::~HTTPFixedLengthInputStream() } -void* HTTPFixedLengthInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPFixedLengthInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPFixedLengthOutputStream // - -Poco::MemoryPool HTTPFixedLengthOutputStream::_pool(sizeof(HTTPFixedLengthOutputStream)); - - HTTPFixedLengthOutputStream::HTTPFixedLengthOutputStream(HTTPSession& session, HTTPFixedLengthStreamBuf::ContentLength length): HTTPFixedLengthIOS(session, length, std::ios::out), std::ostream(&_buf) @@ -163,23 +137,4 @@ HTTPFixedLengthOutputStream::~HTTPFixedLengthOutputStream() } -void* HTTPFixedLengthOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPFixedLengthOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPHeaderStream.cpp b/base/poco/Net/src/HTTPHeaderStream.cpp index 8e0091fcbe3..39b9007062d 100644 --- a/base/poco/Net/src/HTTPHeaderStream.cpp +++ b/base/poco/Net/src/HTTPHeaderStream.cpp @@ -26,7 +26,7 @@ namespace Net { HTTPHeaderStreamBuf::HTTPHeaderStreamBuf(HTTPSession& session, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _end(false) { @@ -101,10 +101,6 @@ HTTPHeaderStreamBuf* HTTPHeaderIOS::rdbuf() // HTTPHeaderInputStream // - -Poco::MemoryPool HTTPHeaderInputStream::_pool(sizeof(HTTPHeaderInputStream)); - - HTTPHeaderInputStream::HTTPHeaderInputStream(HTTPSession& session): HTTPHeaderIOS(session, std::ios::in), std::istream(&_buf) @@ -116,34 +112,10 @@ HTTPHeaderInputStream::~HTTPHeaderInputStream() { } - -void* HTTPHeaderInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPHeaderInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPHeaderOutputStream // - -Poco::MemoryPool HTTPHeaderOutputStream::_pool(sizeof(HTTPHeaderOutputStream)); - - HTTPHeaderOutputStream::HTTPHeaderOutputStream(HTTPSession& session): HTTPHeaderIOS(session, std::ios::out), std::ostream(&_buf) @@ -155,24 +127,4 @@ HTTPHeaderOutputStream::~HTTPHeaderOutputStream() { } - -void* HTTPHeaderOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPHeaderOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index cb6fdc25e9a..1e15b24ddae 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -13,8 +13,8 @@ #include "Poco/Net/HTTPSession.h" -#include "Poco/Net/HTTPBufferAllocator.h" #include "Poco/Net/NetException.h" +#include "Poco/Net/HTTPBasicStreamBuf.h" #include @@ -70,7 +70,7 @@ HTTPSession::~HTTPSession() { try { - if (_pBuffer) HTTPBufferAllocator::deallocate(_pBuffer, HTTPBufferAllocator::BUFFER_SIZE); + if (_pBuffer) delete[] _pBuffer; } catch (...) { @@ -177,10 +177,10 @@ void HTTPSession::refill() { if (!_pBuffer) { - _pBuffer = HTTPBufferAllocator::allocate(HTTPBufferAllocator::BUFFER_SIZE); + _pBuffer = new char[HTTP_DEFAULT_BUFFER_SIZE]; } _pCurrent = _pEnd = _pBuffer; - int n = receive(_pBuffer, HTTPBufferAllocator::BUFFER_SIZE); + int n = receive(_pBuffer, HTTP_DEFAULT_BUFFER_SIZE); _pEnd += n; } diff --git a/base/poco/Net/src/HTTPStream.cpp b/base/poco/Net/src/HTTPStream.cpp index 4acb881c4f3..c2f27600569 100644 --- a/base/poco/Net/src/HTTPStream.cpp +++ b/base/poco/Net/src/HTTPStream.cpp @@ -26,7 +26,7 @@ namespace Net { HTTPStreamBuf::HTTPStreamBuf(HTTPSession& session, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _mode(mode) { @@ -96,10 +96,6 @@ HTTPStreamBuf* HTTPIOS::rdbuf() // HTTPInputStream // - -Poco::MemoryPool HTTPInputStream::_pool(sizeof(HTTPInputStream)); - - HTTPInputStream::HTTPInputStream(HTTPSession& session): HTTPIOS(session, std::ios::in), std::istream(&_buf) @@ -112,33 +108,11 @@ HTTPInputStream::~HTTPInputStream() } -void* HTTPInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPOutputStream // -Poco::MemoryPool HTTPOutputStream::_pool(sizeof(HTTPOutputStream)); - - HTTPOutputStream::HTTPOutputStream(HTTPSession& session): HTTPIOS(session, std::ios::out), std::ostream(&_buf) @@ -150,24 +124,4 @@ HTTPOutputStream::~HTTPOutputStream() { } - -void* HTTPOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net From a74bc6190da07904116ffa36f6ee1340afb74c91 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 3 Jul 2023 15:46:30 +0200 Subject: [PATCH 26/31] Implement suggestion --- base/poco/Net/include/Poco/Net/HTTPSession.h | 2 +- base/poco/Net/src/HTTPSession.cpp | 16 ++++------------ 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index d0045025f5f..934b34be5d5 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -192,7 +192,7 @@ namespace Net HTTPSession & operator=(const HTTPSession &); StreamSocket _socket; - char * _pBuffer; + std::unique_ptr _pBuffer; char * _pCurrent; char * _pEnd; bool _keepAlive; diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 1e15b24ddae..d2663baaf9f 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -68,14 +68,6 @@ HTTPSession::HTTPSession(const StreamSocket& socket, bool keepAlive): HTTPSession::~HTTPSession() { - try - { - if (_pBuffer) delete[] _pBuffer; - } - catch (...) - { - poco_unexpected(); - } try { close(); @@ -177,10 +169,10 @@ void HTTPSession::refill() { if (!_pBuffer) { - _pBuffer = new char[HTTP_DEFAULT_BUFFER_SIZE]; + _pBuffer = std::make_unique(HTTP_DEFAULT_BUFFER_SIZE); } - _pCurrent = _pEnd = _pBuffer; - int n = receive(_pBuffer, HTTP_DEFAULT_BUFFER_SIZE); + _pCurrent = _pEnd = _pBuffer.get(); + int n = receive(_pBuffer.get(), HTTP_DEFAULT_BUFFER_SIZE); _pEnd += n; } @@ -199,7 +191,7 @@ void HTTPSession::connect(const SocketAddress& address) _socket.setNoDelay(true); // There may be leftover data from a previous (failed) request in the buffer, // so we clear it. - _pCurrent = _pEnd = _pBuffer; + _pCurrent = _pEnd = _pBuffer.get(); } From e99e0ebddb712ab40e22be4ab74ebd2ae3786ba0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 4 Jul 2023 13:26:55 +0300 Subject: [PATCH 27/31] Update 01320_create_sync_race_condition_zookeeper.sh (#51742) --- .../01320_create_sync_race_condition_zookeeper.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index ef45e8e63bc..aee69e64b1b 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -12,7 +12,10 @@ $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 --query "CREATE DATABA function thread1() { - while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done + while true; do + $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; + DROP TABLE test_01320.r;" 2>&1 | grep -F "Code:" | grep -v "UNKNOWN_DATABASE" + done } function thread2() From 35c13219520b8b41c633f41423af91b9fda898e1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jul 2023 12:35:16 +0200 Subject: [PATCH 28/31] Version mypy=1.4.1 falsly reports unused ignore comment --- docker/test/style/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 746cc7bb2d5..2aa0b1a62d6 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -18,7 +18,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip From 463c2ba5ce13a6839dfaf12223826b9e90f143c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 14:07:10 +0300 Subject: [PATCH 29/31] Revert "Add documentation for building in docker" --- docker/packager/README.md | 2 +- docker/packager/packager | 16 ++++++++-------- docs/en/development/build.md | 14 -------------- tests/ci/build_check.py | 6 +++--- tests/ci/build_report_check.py | 4 ++-- tests/ci/ci_config.py | 34 +++++++++++++++++----------------- tests/ci/report.py | 6 +++--- 7 files changed, 34 insertions(+), 48 deletions(-) diff --git a/docker/packager/README.md b/docker/packager/README.md index 3a91f9a63f0..a78feb8d7fc 100644 --- a/docker/packager/README.md +++ b/docker/packager/README.md @@ -6,7 +6,7 @@ Usage: Build deb package with `clang-14` in `debug` mode: ``` $ mkdir deb/test_output -$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --debug-build +$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --build-type=debug $ ls -l deb/test_output -rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb -rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb diff --git a/docker/packager/packager b/docker/packager/packager index 3c3304165b3..1b3df858cd2 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -112,12 +112,12 @@ def run_docker_image_with_env( subprocess.check_call(cmd, shell=True) -def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bool: - return not debug_build and package_type == "deb" and sanitizer == "" +def is_release_build(build_type: str, package_type: str, sanitizer: str) -> bool: + return build_type == "" and package_type == "deb" and sanitizer == "" def parse_env_variables( - debug_build: bool, + build_type: str, compiler: str, sanitizer: str, package_type: str, @@ -233,7 +233,7 @@ def parse_env_variables( build_target = ( f"{build_target} clickhouse-odbc-bridge clickhouse-library-bridge" ) - if is_release_build(debug_build, package_type, sanitizer): + if is_release_build(build_type, package_type, sanitizer): cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON") result.append("WITH_PERFORMANCE=1") if is_cross_arm: @@ -253,8 +253,8 @@ def parse_env_variables( if sanitizer: result.append(f"SANITIZER={sanitizer}") - if debug_build: - result.append("BUILD_TYPE=DEBUG") + if build_type: + result.append(f"BUILD_TYPE={build_type.capitalize()}") else: result.append("BUILD_TYPE=None") @@ -359,7 +359,7 @@ def parse_args() -> argparse.Namespace: help="ClickHouse git repository", ) parser.add_argument("--output-dir", type=dir_name, required=True) - parser.add_argument("--debug-build", action="store_true") + parser.add_argument("--build-type", choices=("debug", ""), default="") parser.add_argument( "--compiler", @@ -464,7 +464,7 @@ def main(): build_image(image_with_version, dockerfile) env_prepared = parse_env_variables( - args.debug_build, + args.build_type, args.compiler, args.sanitizer, args.package_type, diff --git a/docs/en/development/build.md b/docs/en/development/build.md index ab47ee8aac5..83a4550df88 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -13,20 +13,6 @@ Supported platforms: - AArch64 - Power9 (experimental) -## Building in docker -We use the docker image `clickhouse/binary-builder` for our CI builds. It contains everything necessary to build the binary and packages. There is a script `docker/packager/packager` to ease the image usage: - -```bash -# define a directory for the output artifacts -output_dir="build_results" -# a simplest build -./docker/packager/packager --package-type=binary --output-dir "$output_dir" -# build debian packages -./docker/packager/packager --package-type=deb --output-dir "$output_dir" -# by default, debian packages use thin LTO, so we can override it to speed up the build -CMAKE_FLAGS='-DENABLE_THINLTO=' ./docker/packager/packager --package-type=deb --output-dir "$output_dir" -``` - ## Building on Ubuntu The following tutorial is based on Ubuntu Linux. diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 2a636faf967..35b98a7c3bb 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -45,7 +45,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: return False if build_config["sanitizer"] != "": return True - if build_config["debug_build"]: + if build_config["build_type"] != "": return True return False @@ -66,8 +66,8 @@ def get_packager_cmd( f"--package-type={package_type} --compiler={comp}" ) - if build_config["debug_build"]: - cmd += " --debug-build" + if build_config["build_type"]: + cmd += f" --build-type={build_config['build_type']}" if build_config["sanitizer"]: cmd += f" --sanitizer={build_config['sanitizer']}" if build_config["tidy"] == "enable": diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 295b6cf9740..1362f3c8934 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -70,7 +70,7 @@ def get_failed_report( message = f"{job_name} failed" build_result = BuildResult( compiler="unknown", - debug_build=False, + build_type="unknown", sanitizer="unknown", status=message, elapsed_seconds=0, @@ -85,7 +85,7 @@ def process_report( build_config = build_report["build_config"] build_result = BuildResult( compiler=build_config["compiler"], - debug_build=build_config["debug_build"], + build_type=build_config["build_type"], sanitizer=build_config["sanitizer"], status="success" if build_report["status"] else "failure", elapsed_seconds=build_report["elapsed_seconds"], diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 875c5a3c8bd..c680b5810fc 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -10,7 +10,7 @@ CI_CONFIG = { "build_config": { "package_release": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "deb", "static_binary_name": "amd64", @@ -21,7 +21,7 @@ CI_CONFIG = { }, "coverity": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "coverity", "tidy": "disable", @@ -31,7 +31,7 @@ CI_CONFIG = { }, "package_aarch64": { "compiler": "clang-16-aarch64", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "deb", "static_binary_name": "aarch64", @@ -42,7 +42,7 @@ CI_CONFIG = { }, "package_asan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "address", "package_type": "deb", "tidy": "disable", @@ -51,7 +51,7 @@ CI_CONFIG = { }, "package_ubsan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "undefined", "package_type": "deb", "tidy": "disable", @@ -60,7 +60,7 @@ CI_CONFIG = { }, "package_tsan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "thread", "package_type": "deb", "tidy": "disable", @@ -69,7 +69,7 @@ CI_CONFIG = { }, "package_msan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "memory", "package_type": "deb", "tidy": "disable", @@ -78,7 +78,7 @@ CI_CONFIG = { }, "package_debug": { "compiler": "clang-16", - "debug_build": True, + "build_type": "debug", "sanitizer": "", "package_type": "deb", "tidy": "disable", @@ -87,7 +87,7 @@ CI_CONFIG = { }, "binary_release": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -96,7 +96,7 @@ CI_CONFIG = { }, "binary_tidy": { "compiler": "clang-16", - "debug_build": True, + "build_type": "debug", "sanitizer": "", "package_type": "binary", "static_binary_name": "debug-amd64", @@ -106,7 +106,7 @@ CI_CONFIG = { }, "binary_darwin": { "compiler": "clang-16-darwin", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "macos", @@ -116,7 +116,7 @@ CI_CONFIG = { }, "binary_aarch64": { "compiler": "clang-16-aarch64", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -125,7 +125,7 @@ CI_CONFIG = { }, "binary_aarch64_v80compat": { "compiler": "clang-16-aarch64-v80compat", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "aarch64v80compat", @@ -135,7 +135,7 @@ CI_CONFIG = { }, "binary_freebsd": { "compiler": "clang-16-freebsd", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "freebsd", @@ -145,7 +145,7 @@ CI_CONFIG = { }, "binary_darwin_aarch64": { "compiler": "clang-16-darwin-aarch64", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "macos-aarch64", @@ -155,7 +155,7 @@ CI_CONFIG = { }, "binary_ppc64le": { "compiler": "clang-16-ppc64le", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "powerpc64le", @@ -165,7 +165,7 @@ CI_CONFIG = { }, "binary_amd64_compat": { "compiler": "clang-16-amd64-compat", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "amd64compat", diff --git a/tests/ci/report.py b/tests/ci/report.py index 0f84fbcaeb2..a9014acec12 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -239,7 +239,7 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes @dataclass class BuildResult: compiler: str - debug_build: bool + build_type: str sanitizer: str status: str elapsed_seconds: int @@ -484,8 +484,8 @@ def create_build_html_report( ): row = "" row += f"{build_result.compiler}" - if build_result.debug_build: - row += "debug" + if build_result.build_type: + row += f"{build_result.build_type}" else: row += "relwithdebuginfo" if build_result.sanitizer: From 5230503651e091f4bf1944a1176aeac47c834907 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 4 Jul 2023 12:59:25 +0000 Subject: [PATCH 30/31] remove unused type: ignore --- tests/ci/commit_status_helper.py | 2 +- tests/ci/github_helper.py | 2 +- tests/ci/merge_pr.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 4437ddc52c6..efe149b0aa4 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -7,7 +7,7 @@ from typing import Dict, List, Literal, Optional, Union import logging from github import Github -from github.GithubObject import _NotSetType, NotSet as NotSet # type: ignore +from github.GithubObject import _NotSetType, NotSet as NotSet from github.Commit import Commit from github.CommitStatus import CommitStatus from github.IssueComment import IssueComment diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 834c8247cb8..f7256e523b1 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -111,7 +111,7 @@ class GitHub(github.Github): # See https://github.com/PyGithub/PyGithub/issues/2202, # obj._rawData doesn't spend additional API requests # pylint: disable=protected-access - repo_url = issue._rawData["repository_url"] # type: ignore + repo_url = issue._rawData["repository_url"] if repo_url not in repos: repos[repo_url] = issue.repository prs.append( diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 2d6d81a152a..14844ed9b25 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -154,7 +154,7 @@ def get_workflows_for_head(repo: Repository, head_sha: str) -> List[WorkflowRun] return list( PaginatedList( WorkflowRun, - repo._requester, # type:ignore # pylint:disable=protected-access + repo._requester, # pylint:disable=protected-access f"{repo.url}/actions/runs", {"head_sha": head_sha}, list_item="workflow_runs", From 3cb459bd04d141ca0fffe7c1f6c389e4be434167 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 4 Jul 2023 15:50:56 +0000 Subject: [PATCH 31/31] Docs: Fix description of output field NON_UNIQUE for statement SHOW INDEXES --- docs/en/sql-reference/statements/show.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index f96eb55aa45..336b93db9d5 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -283,7 +283,7 @@ The optional keyword `EXTENDED` currently has no effect, it only exists for MySQ `SHOW INDEX` produces a result table with the following structure: - table - The name of the table (String) -- non_unique - 0 if the index can contain duplicates, 1 otherwise (UInt8) +- non_unique - 0 if the index cannot contain duplicates, 1 otherwise (UInt8) - key_name - The name of the index, `PRIMARY` if the index is a primary key index (String) - seq_in_index - Currently unused - column_name - Currently unused