Fix memory leak

This commit is contained in:
kssenii 2023-01-31 16:38:35 +01:00
parent f497e3bcbe
commit ad34ff24dc
4 changed files with 19 additions and 19 deletions

View File

@ -412,7 +412,7 @@ KeyTransactionPtr FileCache::createKeyTransaction(const Key & key, KeyNotFoundPo
else if (!it->second || !it->second->guard)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trash in metadata");
return std::make_unique<KeyTransaction>(key, it->second, cleanup_keys_metadata_queue, this);
return std::make_unique<KeyTransaction>(key, *it->second, cleanup_keys_metadata_queue, this);
}
FileSegmentsHolderPtr FileCache::set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings)
@ -553,8 +553,8 @@ FileCache::KeyMetadata::iterator FileCache::addCell(
result_state = state;
}
auto file_segment = std::make_shared<FileSegment>(
offset, size, key, key_transaction.getCreator(), this, result_state, settings);
auto creator = key_transaction.getCreator();
auto file_segment = std::make_shared<FileSegment>(offset, size, key, std::move(creator), this, result_state, settings);
std::optional<LockedCachePriority> locked_queue(lock ? LockedCachePriority(*lock, *main_priority) : std::optional<LockedCachePriority>{});
@ -845,13 +845,12 @@ void FileCache::removeAllReleasable()
KeyTransaction::KeyTransaction(
const Key & key_,
FileCache::KeyMetadataPtr offsets_,
FileCache::KeyMetadata & offsets_,
KeysQueuePtr cleanup_keys_metadata_queue_,
const FileCache * cache_)
: key(key_)
, cache(cache_)
, guard(offsets_->guard)
, lock(guard->lock())
, lock(offsets_.guard->lock())
, offsets(offsets_)
, cleanup_keys_metadata_queue(cleanup_keys_metadata_queue_)
, log(&Poco::Logger::get("KeyTransaction"))
@ -885,7 +884,7 @@ void KeyTransaction::remove(
log, "Remove from cache. Key: {}, offset: {}",
key.toString(), offset);
auto * cell = offsets->get(offset);
auto * cell = offsets.get(offset);
if (cell->queue_iterator)
LockedCachePriorityIterator(cache_lock, cell->queue_iterator).remove();
@ -893,7 +892,7 @@ void KeyTransaction::remove(
const auto cache_file_path = cell->file_segment->getPathInLocalCache();
cell->file_segment->detach(segment_lock, *this);
offsets->erase(offset);
offsets.erase(offset);
if (fs::exists(cache_file_path))
{
@ -919,7 +918,7 @@ void KeyTransaction::cleanupKeyDirectory() const
return;
/// Someone might still need this directory.
if (!offsets->empty())
if (!offsets.empty())
return;
/// Now `offsets` empty and the key lock is still locked.
@ -928,7 +927,7 @@ void KeyTransaction::cleanupKeyDirectory() const
fs::path key_path = cache->getPathInLocalCache(key);
if (fs::exists(key_path))
{
offsets->created_base_directory = false;
offsets.created_base_directory = false;
fs::remove_all(key_path);
}
cleanup_keys_metadata_queue->add(key);
@ -1066,7 +1065,7 @@ void KeyTransaction::reduceSizeToDownloaded(
* because of no space left in cache, we need to be able to cut cell's size to downloaded_size.
*/
auto * cell = offsets->get(offset);
auto * cell = offsets.get(offset);
const auto & file_segment = cell->file_segment;
size_t downloaded_size = file_segment->downloaded_size;

View File

@ -384,7 +384,7 @@ struct KeyTransactionCreator
{
KeyTransactionCreator(
const FileCacheKey & key_,
FileCache::KeyMetadataPtr offsets_,
FileCache::KeyMetadata & offsets_,
KeysQueuePtr cleanup_keys_metadata_queue_,
const FileCache * cache_)
: key(key_), offsets(offsets_), cleanup_keys_metadata_queue(cleanup_keys_metadata_queue_), cache(cache_) {}
@ -392,7 +392,7 @@ struct KeyTransactionCreator
KeyTransactionPtr create();
FileCacheKey key;
FileCache::KeyMetadataPtr offsets;
FileCache::KeyMetadata & offsets;
KeysQueuePtr cleanup_keys_metadata_queue;
const FileCache * cache;
};
@ -404,7 +404,7 @@ struct KeyTransaction : private boost::noncopyable
KeyTransaction(
const Key & key_,
FileCache::KeyMetadataPtr offsets_,
FileCache::KeyMetadata & offsets_,
KeysQueuePtr cleanup_keys_metadata_queue_,
const FileCache * cache_);
@ -420,8 +420,8 @@ struct KeyTransaction : private boost::noncopyable
bool isLastHolder(size_t offset);
FileCache::KeyMetadata & getOffsets() { return *offsets; }
const FileCache::KeyMetadata & getOffsets() const { return *offsets; }
FileCache::KeyMetadata & getOffsets() { return offsets; }
const FileCache::KeyMetadata & getOffsets() const { return offsets; }
std::vector<size_t> delete_offsets;
@ -432,10 +432,8 @@ private:
Key key;
const FileCache * cache;
KeyGuardPtr guard;
KeyGuard::Lock lock;
FileCache::KeyMetadataPtr offsets;
FileCache::KeyMetadata & offsets;
KeysQueuePtr cleanup_keys_metadata_queue;
Poco::Logger * log;

View File

@ -129,6 +129,8 @@ public:
State download_state_,
const CreateFileSegmentSettings & create_settings);
~FileSegment() = default;
State state() const;
static String stateToString(FileSegment::State state);

View File

@ -1,3 +1,4 @@
#pragma once
#include <Interpreters/Cache/IFileCachePriority.h>