Better names for some methods

This commit is contained in:
kssenii 2023-03-22 17:34:29 +01:00
parent 9b9cf13607
commit 5b7f214ea9
5 changed files with 39 additions and 36 deletions

View File

@ -589,6 +589,16 @@ void FileCache::iterateCacheAndCollectKeyLocks(
}); });
} }
void FileCache::removeFileSegment(LockedKey & locked_key, FileSegmentPtr file_segment, const CacheGuard::Lock & cache_lock)
{
/// FIXME:
/// We must hold pointer to file segment while removing it (because we remove file segment under file segment lock).
/// But this should not be obligatory.
chassert(file_segment->key() == locked_key.getKey());
locked_key.removeFileSegment(file_segment->offset(), file_segment->lock(), cache_lock);
}
bool FileCache::tryReserveImpl( bool FileCache::tryReserveImpl(
IFileCachePriority & priority_queue, IFileCachePriority & priority_queue,
const Key & key, const Key & key,
@ -682,7 +692,7 @@ bool FileCache::tryReserveImpl(
{ {
remove_current_it = true; remove_current_it = true;
file_segment_metadata->queue_iterator = {}; file_segment_metadata->queue_iterator = {};
current_locked_key.remove(file_segment, cache_lock); removeFileSegment(current_locked_key, file_segment, cache_lock);
break; break;
} }
} }
@ -708,7 +718,7 @@ bool FileCache::tryReserveImpl(
for (const auto & offset_to_delete : offsets_to_delete) for (const auto & offset_to_delete : offsets_to_delete)
{ {
auto * file_segment_metadata = current_locked_key->getKeyMetadata().getByOffset(offset_to_delete); auto * file_segment_metadata = current_locked_key->getKeyMetadata().getByOffset(offset_to_delete);
current_locked_key->remove(file_segment_metadata->file_segment, cache_lock); removeFileSegment(*current_locked_key, file_segment_metadata->file_segment, cache_lock);
if (query_context) if (query_context)
query_context->remove(key, offset); query_context->remove(key, offset);
} }
@ -780,7 +790,7 @@ void FileCache::removeKeyIfExists(const Key & key)
if (!file_segment_metadata->releasable()) if (!file_segment_metadata->releasable())
continue; continue;
locked_key->remove(file_segment_metadata->file_segment, lock); removeFileSegment(*locked_key, file_segment_metadata->file_segment, lock);
} }
} }
} }
@ -809,7 +819,7 @@ void FileCache::removeAllReleasable()
if (file_segment_metadata->releasable()) if (file_segment_metadata->releasable())
{ {
file_segment_metadata->queue_iterator = {}; file_segment_metadata->queue_iterator = {};
locked_key->remove(file_segment_metadata->file_segment, lock); removeFileSegment(*locked_key, file_segment_metadata->file_segment, lock);
return IterationResult::REMOVE_AND_CONTINUE; return IterationResult::REMOVE_AND_CONTINUE;
} }
return IterationResult::CONTINUE; return IterationResult::CONTINUE;
@ -1026,10 +1036,10 @@ LockedKeyPtr FileCache::createLockedKey(const Key & key, KeyNotFoundPolicy key_n
return nullptr; return nullptr;
it = metadata.emplace(key, std::make_shared<KeyMetadata>()).first; it = metadata.emplace(key, std::make_shared<KeyMetadata>()).first;
return std::make_unique<LockedKey>(key, *it->second, it->second->lock(), cleanup_keys_metadata_queue, getPathInLocalCache(key)); return std::make_unique<LockedKey>(key, *it->second, it->second->lock(), getPathInLocalCache(key), cleanup_keys_metadata_queue);
} }
return std::make_unique<LockedKey>(key, *key_metadata, std::move(key_lock), cleanup_keys_metadata_queue, getPathInLocalCache(key)); return std::make_unique<LockedKey>(key, *key_metadata, std::move(key_lock), getPathInLocalCache(key), cleanup_keys_metadata_queue);
} }
LockedKeyPtr FileCache::createLockedKey(const Key & key, KeyMetadata & key_metadata) const LockedKeyPtr FileCache::createLockedKey(const Key & key, KeyMetadata & key_metadata) const
@ -1037,7 +1047,7 @@ LockedKeyPtr FileCache::createLockedKey(const Key & key, KeyMetadata & key_metad
auto key_lock = key_metadata.lock(); auto key_lock = key_metadata.lock();
if (key_metadata.removed) if (key_metadata.removed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot lock key: it was removed from cache"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot lock key: it was removed from cache");
return std::make_unique<LockedKey>(key, key_metadata, std::move(key_lock), cleanup_keys_metadata_queue, getPathInLocalCache(key)); return std::make_unique<LockedKey>(key, key_metadata, std::move(key_lock), getPathInLocalCache(key), cleanup_keys_metadata_queue);
} }
void FileCache::iterateCacheMetadata(const CacheMetadataGuard::Lock & lock, std::function<void(KeyMetadata &)> && func) void FileCache::iterateCacheMetadata(const CacheMetadataGuard::Lock & lock, std::function<void(KeyMetadata &)> && func)

View File

@ -245,6 +245,8 @@ private:
void assertCacheCorrectness(); void assertCacheCorrectness();
void assertCacheCorrectness(const CacheGuard::Lock & cache_lock, const CacheMetadataGuard::Lock & metadata_lock); void assertCacheCorrectness(const CacheGuard::Lock & cache_lock, const CacheMetadataGuard::Lock & metadata_lock);
static void removeFileSegment(LockedKey & locked_key, FileSegmentPtr file_segment, const CacheGuard::Lock &);
}; };
} }

View File

@ -56,7 +56,7 @@ FileSegment::FileSegment(
break; break;
} }
/// DOWNLOADED is used either on initial cache metadata load into memory on server startup /// DOWNLOADED is used either on initial cache metadata load into memory on server startup
/// or on reduceSizeToDownloaded() -- when file segment object is updated. /// or on shrinkFileSegmentToDownloadedSize() -- when file segment object is updated.
case (State::DOWNLOADED): case (State::DOWNLOADED):
{ {
reserved_size = downloaded_size = size_; reserved_size = downloaded_size = size_;
@ -254,7 +254,7 @@ FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader()
assert(download_state != State::DETACHED); assert(download_state != State::DETACHED);
bool is_last_holder = locked_key->isLastHolder(offset()); bool is_last_holder = locked_key->isLastOwnerOfFileSegment(offset());
if (!downloader_id.empty() || !is_last_holder) if (!downloader_id.empty() || !is_last_holder)
return nullptr; return nullptr;
@ -445,7 +445,7 @@ bool FileSegment::reserve(size_t size_to_reserve)
{ {
/// No lock is required because reserved size is always /// No lock is required because reserved size is always
/// mananaged (read/modified) by the downloader only /// mananaged (read/modified) by the downloader only
/// or in isLastHolder() case. /// or in isLastOwnerOfFileSegment() case.
/// It is made atomic because of getInfoForLog. /// It is made atomic because of getInfoForLog.
reserved_size += size_to_reserve; reserved_size += size_to_reserve;
} }
@ -534,7 +534,7 @@ void FileSegment::completeUnlocked(LockedKey & locked_key, const CacheGuard::Loc
return; return;
const bool is_downloader = isDownloaderUnlocked(segment_lock); const bool is_downloader = isDownloaderUnlocked(segment_lock);
const bool is_last_holder = locked_key.isLastHolder(offset()); const bool is_last_holder = locked_key.isLastOwnerOfFileSegment(offset());
const size_t current_downloaded_size = getDownloadedSize(true); const size_t current_downloaded_size = getDownloadedSize(true);
SCOPE_EXIT({ SCOPE_EXIT({
@ -567,7 +567,7 @@ void FileSegment::completeUnlocked(LockedKey & locked_key, const CacheGuard::Loc
LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock));
detach(segment_lock, locked_key); detach(segment_lock, locked_key);
setDownloadState(State::DETACHED, segment_lock); setDownloadState(State::DETACHED, segment_lock);
locked_key.remove(offset(), segment_lock, cache_lock); locked_key.removeFileSegment(offset(), segment_lock, cache_lock);
return; return;
} }
@ -598,7 +598,7 @@ void FileSegment::completeUnlocked(LockedKey & locked_key, const CacheGuard::Loc
LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString()); LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString());
setDownloadState(State::DETACHED, segment_lock); setDownloadState(State::DETACHED, segment_lock);
locked_key.remove(offset(), segment_lock, cache_lock); locked_key.removeFileSegment(offset(), segment_lock, cache_lock);
} }
else else
{ {
@ -616,7 +616,7 @@ void FileSegment::completeUnlocked(LockedKey & locked_key, const CacheGuard::Loc
/// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state, /// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state,
/// because otherwise an invariant that getOrSet() returns a contiguous range of file segments will be broken /// because otherwise an invariant that getOrSet() returns a contiguous range of file segments will be broken
/// (this will be crucial for other file segment holder, not for current one). /// (this will be crucial for other file segment holder, not for current one).
locked_key.reduceSizeToDownloaded(offset(), segment_lock, cache_lock); locked_key.shrinkFileSegmentToDownloadedSize(offset(), segment_lock, cache_lock);
} }
detachAssumeStateFinalized(segment_lock); detachAssumeStateFinalized(segment_lock);

View File

@ -18,8 +18,8 @@ LockedKey::LockedKey(
const FileCacheKey & key_, const FileCacheKey & key_,
KeyMetadata & key_metadata_, KeyMetadata & key_metadata_,
KeyGuard::Lock && lock_, KeyGuard::Lock && lock_,
KeysQueuePtr cleanup_keys_metadata_queue_, const std::string & key_path_,
const std::string & key_path_) KeysQueuePtr cleanup_keys_metadata_queue_)
: key(key_) : key(key_)
, key_path(key_path_) , key_path(key_path_)
, lock(std::move(lock_)) , lock(std::move(lock_))
@ -34,20 +34,13 @@ LockedKey::~LockedKey()
removeKeyIfEmpty(); removeKeyIfEmpty();
} }
void LockedKey::remove(FileSegmentPtr file_segment, const CacheGuard::Lock & cache_lock) bool LockedKey::isLastOwnerOfFileSegment(size_t offset) const
{
/// We must hold pointer to file segment while removing it.
chassert(file_segment->key() == key);
remove(file_segment->offset(), file_segment->lock(), cache_lock);
}
bool LockedKey::isLastHolder(size_t offset) const
{ {
const auto * file_segment_metadata = key_metadata.getByOffset(offset); const auto * file_segment_metadata = key_metadata.getByOffset(offset);
return file_segment_metadata->file_segment.use_count() == 2; return file_segment_metadata->file_segment.use_count() == 2;
} }
void LockedKey::remove( void LockedKey::removeFileSegment(
size_t offset, size_t offset,
const FileSegmentGuard::Lock & segment_lock, const FileSegmentGuard::Lock & segment_lock,
const CacheGuard::Lock & cache_lock) const CacheGuard::Lock & cache_lock)
@ -82,7 +75,7 @@ void LockedKey::remove(
} }
} }
void LockedKey::reduceSizeToDownloaded( void LockedKey::shrinkFileSegmentToDownloadedSize(
size_t offset, size_t offset,
const FileSegmentGuard::Lock & segment_lock, const FileSegmentGuard::Lock & segment_lock,
const CacheGuard::Lock & cache_lock) const CacheGuard::Lock & cache_lock)

View File

@ -27,23 +27,21 @@ struct LockedKey : private boost::noncopyable
const FileCacheKey & key_, const FileCacheKey & key_,
KeyMetadata & key_metadata_, KeyMetadata & key_metadata_,
KeyGuard::Lock && key_lock_, KeyGuard::Lock && key_lock_,
KeysQueuePtr cleanup_keys_metadata_queue_, const std::string & key_path_,
const std::string & key_path_); KeysQueuePtr cleanup_keys_metadata_queue_);
~LockedKey(); ~LockedKey();
void reduceSizeToDownloaded(size_t offset, const FileSegmentGuard::Lock &, const CacheGuard::Lock &); const FileCacheKey & getKey() const { return key; }
void remove(FileSegmentPtr file_segment, const CacheGuard::Lock &);
void remove(size_t offset, const FileSegmentGuard::Lock &, const CacheGuard::Lock &);
bool isLastHolder(size_t offset) const;
const KeyMetadata & getKeyMetadata() const { return key_metadata; } const KeyMetadata & getKeyMetadata() const { return key_metadata; }
KeyMetadata & getKeyMetadata() { return key_metadata; } KeyMetadata & getKeyMetadata() { return key_metadata; }
const FileCacheKey & getKey() const { return key; } void removeFileSegment(size_t offset, const FileSegmentGuard::Lock &, const CacheGuard::Lock &);
void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &, const CacheGuard::Lock &);
bool isLastOwnerOfFileSegment(size_t offset) const;
private: private:
void removeKeyIfEmpty() const; void removeKeyIfEmpty() const;