mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #52778 from kssenii/revert-revert-drop-cache
Revert revert of system drop filesystem cache by key
This commit is contained in:
commit
621d852289
@ -74,19 +74,22 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
|
|||||||
}
|
}
|
||||||
|
|
||||||
void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog(
|
void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog(
|
||||||
const FileSegment::Range & file_segment_range, CachedOnDiskReadBufferFromFile::ReadType type)
|
const FileSegment & file_segment, CachedOnDiskReadBufferFromFile::ReadType type)
|
||||||
{
|
{
|
||||||
if (!cache_log)
|
if (!cache_log)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
|
const auto range = file_segment.range();
|
||||||
FilesystemCacheLogElement elem
|
FilesystemCacheLogElement elem
|
||||||
{
|
{
|
||||||
.event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()),
|
.event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()),
|
||||||
.query_id = query_id,
|
.query_id = query_id,
|
||||||
.source_file_path = source_file_path,
|
.source_file_path = source_file_path,
|
||||||
.file_segment_range = { file_segment_range.left, file_segment_range.right },
|
.file_segment_range = { range.left, range.right },
|
||||||
.requested_range = { first_offset, read_until_position },
|
.requested_range = { first_offset, read_until_position },
|
||||||
.file_segment_size = file_segment_range.size(),
|
.file_segment_key = file_segment.key().toString(),
|
||||||
|
.file_segment_offset = file_segment.offset(),
|
||||||
|
.file_segment_size = range.size(),
|
||||||
.read_from_cache_attempted = true,
|
.read_from_cache_attempted = true,
|
||||||
.read_buffer_id = current_buffer_id,
|
.read_buffer_id = current_buffer_id,
|
||||||
.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(
|
.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(
|
||||||
@ -495,7 +498,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
|
|||||||
auto completed_range = current_file_segment->range();
|
auto completed_range = current_file_segment->range();
|
||||||
|
|
||||||
if (cache_log)
|
if (cache_log)
|
||||||
appendFilesystemCacheLog(completed_range, read_type);
|
appendFilesystemCacheLog(*current_file_segment, read_type);
|
||||||
|
|
||||||
chassert(file_offset_of_buffer_end > completed_range.right);
|
chassert(file_offset_of_buffer_end > completed_range.right);
|
||||||
|
|
||||||
@ -518,7 +521,7 @@ CachedOnDiskReadBufferFromFile::~CachedOnDiskReadBufferFromFile()
|
|||||||
{
|
{
|
||||||
if (cache_log && file_segments && !file_segments->empty())
|
if (cache_log && file_segments && !file_segments->empty())
|
||||||
{
|
{
|
||||||
appendFilesystemCacheLog(file_segments->front().range(), read_type);
|
appendFilesystemCacheLog(file_segments->front(), read_type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -90,7 +90,7 @@ private:
|
|||||||
|
|
||||||
bool completeFileSegmentAndGetNext();
|
bool completeFileSegmentAndGetNext();
|
||||||
|
|
||||||
void appendFilesystemCacheLog(const FileSegment::Range & file_segment_range, ReadType read_type);
|
void appendFilesystemCacheLog(const FileSegment & file_segment, ReadType read_type);
|
||||||
|
|
||||||
bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment);
|
bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment);
|
||||||
|
|
||||||
|
@ -109,6 +109,8 @@ void ReadBufferFromRemoteFSGather::appendUncachedReadInfo()
|
|||||||
.source_file_path = current_object.remote_path,
|
.source_file_path = current_object.remote_path,
|
||||||
.file_segment_range = { 0, current_object.bytes_size },
|
.file_segment_range = { 0, current_object.bytes_size },
|
||||||
.cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE,
|
.cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE,
|
||||||
|
.file_segment_key = {},
|
||||||
|
.file_segment_offset = {},
|
||||||
.file_segment_size = current_object.bytes_size,
|
.file_segment_size = current_object.bytes_size,
|
||||||
.read_from_cache_attempted = false,
|
.read_from_cache_attempted = false,
|
||||||
};
|
};
|
||||||
|
@ -806,6 +806,13 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size)
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void FileCache::removeKey(const Key & key)
|
||||||
|
{
|
||||||
|
assertInitialized();
|
||||||
|
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
|
||||||
|
locked_key->removeAll();
|
||||||
|
}
|
||||||
|
|
||||||
void FileCache::removeKeyIfExists(const Key & key)
|
void FileCache::removeKeyIfExists(const Key & key)
|
||||||
{
|
{
|
||||||
assertInitialized();
|
assertInitialized();
|
||||||
@ -818,7 +825,14 @@ void FileCache::removeKeyIfExists(const Key & key)
|
|||||||
/// But if we have multiple replicated zero-copy tables on the same server
|
/// But if we have multiple replicated zero-copy tables on the same server
|
||||||
/// it became possible to start removing something from cache when it is used
|
/// it became possible to start removing something from cache when it is used
|
||||||
/// by other "zero-copy" tables. That is why it's not an error.
|
/// by other "zero-copy" tables. That is why it's not an error.
|
||||||
locked_key->removeAllReleasable();
|
locked_key->removeAll(/* if_releasable */true);
|
||||||
|
}
|
||||||
|
|
||||||
|
void FileCache::removeFileSegment(const Key & key, size_t offset)
|
||||||
|
{
|
||||||
|
assertInitialized();
|
||||||
|
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
|
||||||
|
locked_key->removeFileSegment(offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
void FileCache::removePathIfExists(const String & path)
|
void FileCache::removePathIfExists(const String & path)
|
||||||
@ -830,22 +844,12 @@ void FileCache::removeAllReleasable()
|
|||||||
{
|
{
|
||||||
assertInitialized();
|
assertInitialized();
|
||||||
|
|
||||||
auto lock = lockCache();
|
metadata.iterate([](LockedKey & locked_key) { locked_key.removeAll(/* if_releasable */true); });
|
||||||
|
|
||||||
main_priority->iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata)
|
|
||||||
{
|
|
||||||
if (segment_metadata->releasable())
|
|
||||||
{
|
|
||||||
auto file_segment = segment_metadata->file_segment;
|
|
||||||
locked_key.removeFileSegment(file_segment->offset(), file_segment->lock());
|
|
||||||
return PriorityIterationResult::REMOVE_AND_CONTINUE;
|
|
||||||
}
|
|
||||||
return PriorityIterationResult::CONTINUE;
|
|
||||||
}, lock);
|
|
||||||
|
|
||||||
if (stash)
|
if (stash)
|
||||||
{
|
{
|
||||||
/// Remove all access information.
|
/// Remove all access information.
|
||||||
|
auto lock = lockCache();
|
||||||
stash->records.clear();
|
stash->records.clear();
|
||||||
stash->queue->removeAll(lock);
|
stash->queue->removeAll(lock);
|
||||||
}
|
}
|
||||||
@ -914,7 +918,7 @@ void FileCache::loadMetadata()
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto key = Key(unhexUInt<UInt128>(key_directory.filename().string().data()));
|
const auto key = Key::fromKeyString(key_directory.filename().string());
|
||||||
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true);
|
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true);
|
||||||
|
|
||||||
for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it)
|
for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it)
|
||||||
@ -1069,7 +1073,7 @@ FileSegmentsHolderPtr FileCache::getSnapshot()
|
|||||||
FileSegmentsHolderPtr FileCache::getSnapshot(const Key & key)
|
FileSegmentsHolderPtr FileCache::getSnapshot(const Key & key)
|
||||||
{
|
{
|
||||||
FileSegments file_segments;
|
FileSegments file_segments;
|
||||||
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
|
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL);
|
||||||
for (const auto & [_, file_segment_metadata] : *locked_key->getKeyMetadata())
|
for (const auto & [_, file_segment_metadata] : *locked_key->getKeyMetadata())
|
||||||
file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment));
|
file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment));
|
||||||
return std::make_unique<FileSegmentsHolder>(std::move(file_segments));
|
return std::make_unique<FileSegmentsHolder>(std::move(file_segments));
|
||||||
|
@ -83,13 +83,19 @@ public:
|
|||||||
|
|
||||||
FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
|
FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
|
||||||
|
|
||||||
/// Remove files by `key`. Removes files which might be used at the moment.
|
/// Remove file segment by `key` and `offset`. Throws if file segment does not exist.
|
||||||
|
void removeFileSegment(const Key & key, size_t offset);
|
||||||
|
|
||||||
|
/// Remove files by `key`. Throws if key does not exist.
|
||||||
|
void removeKey(const Key & key);
|
||||||
|
|
||||||
|
/// Remove files by `key`.
|
||||||
void removeKeyIfExists(const Key & key);
|
void removeKeyIfExists(const Key & key);
|
||||||
|
|
||||||
/// Removes files by `path`. Removes files which might be used at the moment.
|
/// Removes files by `path`.
|
||||||
void removePathIfExists(const String & path);
|
void removePathIfExists(const String & path);
|
||||||
|
|
||||||
/// Remove files by `key`. Will not remove files which are used at the moment.
|
/// Remove files by `key`.
|
||||||
void removeAllReleasable();
|
void removeAllReleasable();
|
||||||
|
|
||||||
std::vector<String> tryGetCachePaths(const Key & key);
|
std::vector<String> tryGetCachePaths(const Key & key);
|
||||||
|
@ -7,6 +7,10 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
|
}
|
||||||
|
|
||||||
FileCacheKey::FileCacheKey(const std::string & path)
|
FileCacheKey::FileCacheKey(const std::string & path)
|
||||||
: key(sipHash128(path.data(), path.size()))
|
: key(sipHash128(path.data(), path.size()))
|
||||||
@ -28,4 +32,11 @@ FileCacheKey FileCacheKey::random()
|
|||||||
return FileCacheKey(UUIDHelpers::generateV4().toUnderType());
|
return FileCacheKey(UUIDHelpers::generateV4().toUnderType());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
FileCacheKey FileCacheKey::fromKeyString(const std::string & key_str)
|
||||||
|
{
|
||||||
|
if (key_str.size() != 32)
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid cache key hex: {}", key_str);
|
||||||
|
return FileCacheKey(unhexUInt<UInt128>(key_str.data()));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -21,6 +21,8 @@ struct FileCacheKey
|
|||||||
static FileCacheKey random();
|
static FileCacheKey random();
|
||||||
|
|
||||||
bool operator==(const FileCacheKey & other) const { return key == other.key; }
|
bool operator==(const FileCacheKey & other) const { return key == other.key; }
|
||||||
|
|
||||||
|
static FileCacheKey fromKeyString(const std::string & key_str);
|
||||||
};
|
};
|
||||||
|
|
||||||
using FileCacheKeyAndOffset = std::pair<FileCacheKey, size_t>;
|
using FileCacheKeyAndOffset = std::pair<FileCacheKey, size_t>;
|
||||||
|
@ -25,6 +25,7 @@ namespace DB
|
|||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_)
|
FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_)
|
||||||
@ -191,6 +192,8 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
|
|||||||
if (it == end())
|
if (it == end())
|
||||||
{
|
{
|
||||||
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
|
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key);
|
||||||
|
else if (key_not_found_policy == KeyNotFoundPolicy::THROW_LOGICAL)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key);
|
||||||
else if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
|
else if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
|
||||||
return nullptr;
|
return nullptr;
|
||||||
@ -215,6 +218,8 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
|
|||||||
return locked_metadata;
|
return locked_metadata;
|
||||||
|
|
||||||
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
|
if (key_not_found_policy == KeyNotFoundPolicy::THROW)
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key);
|
||||||
|
else if (key_not_found_policy == KeyNotFoundPolicy::THROW_LOGICAL)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key);
|
||||||
|
|
||||||
if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
|
if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL)
|
||||||
@ -561,11 +566,11 @@ bool LockedKey::isLastOwnerOfFileSegment(size_t offset) const
|
|||||||
return file_segment_metadata->file_segment.use_count() == 2;
|
return file_segment_metadata->file_segment.use_count() == 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
void LockedKey::removeAllReleasable()
|
void LockedKey::removeAll(bool if_releasable)
|
||||||
{
|
{
|
||||||
for (auto it = key_metadata->begin(); it != key_metadata->end();)
|
for (auto it = key_metadata->begin(); it != key_metadata->end();)
|
||||||
{
|
{
|
||||||
if (!it->second->releasable())
|
if (if_releasable && !it->second->releasable())
|
||||||
{
|
{
|
||||||
++it;
|
++it;
|
||||||
continue;
|
continue;
|
||||||
@ -586,17 +591,32 @@ void LockedKey::removeAllReleasable()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset)
|
||||||
|
{
|
||||||
|
auto it = key_metadata->find(offset);
|
||||||
|
if (it == key_metadata->end())
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no offset {}", offset);
|
||||||
|
|
||||||
|
auto file_segment = it->second->file_segment;
|
||||||
|
return removeFileSegmentImpl(it, file_segment->lock());
|
||||||
|
}
|
||||||
|
|
||||||
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegmentGuard::Lock & segment_lock)
|
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegmentGuard::Lock & segment_lock)
|
||||||
{
|
{
|
||||||
auto it = key_metadata->find(offset);
|
auto it = key_metadata->find(offset);
|
||||||
if (it == key_metadata->end())
|
if (it == key_metadata->end())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no offset {}", offset);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no offset {}", offset);
|
||||||
|
|
||||||
|
return removeFileSegmentImpl(it, segment_lock);
|
||||||
|
}
|
||||||
|
|
||||||
|
KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock & segment_lock)
|
||||||
|
{
|
||||||
auto file_segment = it->second->file_segment;
|
auto file_segment = it->second->file_segment;
|
||||||
|
|
||||||
LOG_DEBUG(
|
LOG_DEBUG(
|
||||||
key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}",
|
key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}",
|
||||||
getKey(), offset, file_segment->reserved_size);
|
getKey(), file_segment->offset(), file_segment->reserved_size);
|
||||||
|
|
||||||
chassert(file_segment->assertCorrectnessUnlocked(segment_lock));
|
chassert(file_segment->assertCorrectnessUnlocked(segment_lock));
|
||||||
|
|
||||||
|
@ -87,7 +87,7 @@ struct CacheMetadata : public std::unordered_map<FileCacheKey, KeyMetadataPtr>,
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
using Key = FileCacheKey;
|
using Key = FileCacheKey;
|
||||||
using IterateCacheMetadataFunc = std::function<void(const LockedKey &)>;
|
using IterateCacheMetadataFunc = std::function<void(LockedKey &)>;
|
||||||
|
|
||||||
explicit CacheMetadata(const std::string & path_);
|
explicit CacheMetadata(const std::string & path_);
|
||||||
|
|
||||||
@ -106,6 +106,7 @@ public:
|
|||||||
enum class KeyNotFoundPolicy
|
enum class KeyNotFoundPolicy
|
||||||
{
|
{
|
||||||
THROW,
|
THROW,
|
||||||
|
THROW_LOGICAL,
|
||||||
CREATE_EMPTY,
|
CREATE_EMPTY,
|
||||||
RETURN_NULL,
|
RETURN_NULL,
|
||||||
};
|
};
|
||||||
@ -169,9 +170,10 @@ struct LockedKey : private boost::noncopyable
|
|||||||
std::shared_ptr<const KeyMetadata> getKeyMetadata() const { return key_metadata; }
|
std::shared_ptr<const KeyMetadata> getKeyMetadata() const { return key_metadata; }
|
||||||
std::shared_ptr<KeyMetadata> getKeyMetadata() { return key_metadata; }
|
std::shared_ptr<KeyMetadata> getKeyMetadata() { return key_metadata; }
|
||||||
|
|
||||||
void removeAllReleasable();
|
void removeAll(bool if_releasable = true);
|
||||||
|
|
||||||
KeyMetadata::iterator removeFileSegment(size_t offset, const FileSegmentGuard::Lock &);
|
KeyMetadata::iterator removeFileSegment(size_t offset, const FileSegmentGuard::Lock &);
|
||||||
|
KeyMetadata::iterator removeFileSegment(size_t offset);
|
||||||
|
|
||||||
void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &);
|
void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &);
|
||||||
|
|
||||||
@ -188,6 +190,8 @@ struct LockedKey : private boost::noncopyable
|
|||||||
std::string toString() const;
|
std::string toString() const;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
KeyMetadata::iterator removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock &);
|
||||||
|
|
||||||
const std::shared_ptr<KeyMetadata> key_metadata;
|
const std::shared_ptr<KeyMetadata> key_metadata;
|
||||||
KeyGuard::Lock lock; /// `lock` must be destructed before `key_metadata`.
|
KeyGuard::Lock lock; /// `lock` must be destructed before `key_metadata`.
|
||||||
};
|
};
|
||||||
|
@ -40,6 +40,8 @@ NamesAndTypesList FilesystemCacheLogElement::getNamesAndTypes()
|
|||||||
{"source_file_path", std::make_shared<DataTypeString>()},
|
{"source_file_path", std::make_shared<DataTypeString>()},
|
||||||
{"file_segment_range", std::make_shared<DataTypeTuple>(types)},
|
{"file_segment_range", std::make_shared<DataTypeTuple>(types)},
|
||||||
{"total_requested_range", std::make_shared<DataTypeTuple>(types)},
|
{"total_requested_range", std::make_shared<DataTypeTuple>(types)},
|
||||||
|
{"key", std::make_shared<DataTypeString>()},
|
||||||
|
{"offset", std::make_shared<DataTypeUInt64>()},
|
||||||
{"size", std::make_shared<DataTypeUInt64>()},
|
{"size", std::make_shared<DataTypeUInt64>()},
|
||||||
{"read_type", std::make_shared<DataTypeString>()},
|
{"read_type", std::make_shared<DataTypeString>()},
|
||||||
{"read_from_cache_attempted", std::make_shared<DataTypeUInt8>()},
|
{"read_from_cache_attempted", std::make_shared<DataTypeUInt8>()},
|
||||||
@ -60,6 +62,8 @@ void FilesystemCacheLogElement::appendToBlock(MutableColumns & columns) const
|
|||||||
columns[i++]->insert(source_file_path);
|
columns[i++]->insert(source_file_path);
|
||||||
columns[i++]->insert(Tuple{file_segment_range.first, file_segment_range.second});
|
columns[i++]->insert(Tuple{file_segment_range.first, file_segment_range.second});
|
||||||
columns[i++]->insert(Tuple{requested_range.first, requested_range.second});
|
columns[i++]->insert(Tuple{requested_range.first, requested_range.second});
|
||||||
|
columns[i++]->insert(file_segment_key);
|
||||||
|
columns[i++]->insert(file_segment_offset);
|
||||||
columns[i++]->insert(file_segment_size);
|
columns[i++]->insert(file_segment_size);
|
||||||
columns[i++]->insert(typeToString(cache_type));
|
columns[i++]->insert(typeToString(cache_type));
|
||||||
columns[i++]->insert(read_from_cache_attempted);
|
columns[i++]->insert(read_from_cache_attempted);
|
||||||
|
@ -11,16 +11,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
///
|
|
||||||
/// -------- Column --------- Type ------
|
|
||||||
/// | event_date | DateTime |
|
|
||||||
/// | event_time | UInt64 |
|
|
||||||
/// | query_id | String |
|
|
||||||
/// | remote_file_path | String |
|
|
||||||
/// | segment_range | Tuple |
|
|
||||||
/// | read_type | String |
|
|
||||||
/// -------------------------------------
|
|
||||||
///
|
|
||||||
struct FilesystemCacheLogElement
|
struct FilesystemCacheLogElement
|
||||||
{
|
{
|
||||||
enum class CacheType
|
enum class CacheType
|
||||||
@ -39,6 +30,8 @@ struct FilesystemCacheLogElement
|
|||||||
std::pair<size_t, size_t> file_segment_range{};
|
std::pair<size_t, size_t> file_segment_range{};
|
||||||
std::pair<size_t, size_t> requested_range{};
|
std::pair<size_t, size_t> requested_range{};
|
||||||
CacheType cache_type{};
|
CacheType cache_type{};
|
||||||
|
std::string file_segment_key;
|
||||||
|
size_t file_segment_offset;
|
||||||
size_t file_segment_size;
|
size_t file_segment_size;
|
||||||
bool read_from_cache_attempted;
|
bool read_from_cache_attempted;
|
||||||
String read_buffer_id;
|
String read_buffer_id;
|
||||||
|
@ -371,7 +371,18 @@ BlockIO InterpreterSystemQuery::execute()
|
|||||||
else
|
else
|
||||||
{
|
{
|
||||||
auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name).cache;
|
auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name).cache;
|
||||||
cache->removeAllReleasable();
|
if (query.key_to_drop.empty())
|
||||||
|
{
|
||||||
|
cache->removeAllReleasable();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto key = FileCacheKey::fromKeyString(query.key_to_drop);
|
||||||
|
if (query.offset_to_drop.has_value())
|
||||||
|
cache->removeFileSegment(key, query.offset_to_drop.value());
|
||||||
|
else
|
||||||
|
cache->removeKey(key);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -210,7 +210,15 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
|||||||
else if (type == Type::DROP_FILESYSTEM_CACHE)
|
else if (type == Type::DROP_FILESYSTEM_CACHE)
|
||||||
{
|
{
|
||||||
if (!filesystem_cache_name.empty())
|
if (!filesystem_cache_name.empty())
|
||||||
|
{
|
||||||
settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_name;
|
settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_name;
|
||||||
|
if (!key_to_drop.empty())
|
||||||
|
{
|
||||||
|
settings.ostr << (settings.hilite ? hilite_none : "") << " KEY " << key_to_drop;
|
||||||
|
if (offset_to_drop.has_value())
|
||||||
|
settings.ostr << (settings.hilite ? hilite_none : "") << " OFFSET " << offset_to_drop.value();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else if (type == Type::UNFREEZE)
|
else if (type == Type::UNFREEZE)
|
||||||
{
|
{
|
||||||
|
@ -108,6 +108,8 @@ public:
|
|||||||
UInt64 seconds{};
|
UInt64 seconds{};
|
||||||
|
|
||||||
String filesystem_cache_name;
|
String filesystem_cache_name;
|
||||||
|
std::string key_to_drop;
|
||||||
|
std::optional<size_t> offset_to_drop;
|
||||||
|
|
||||||
String backup_name;
|
String backup_name;
|
||||||
|
|
||||||
|
@ -405,7 +405,15 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
|||||||
ParserLiteral path_parser;
|
ParserLiteral path_parser;
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
if (path_parser.parse(pos, ast, expected))
|
if (path_parser.parse(pos, ast, expected))
|
||||||
|
{
|
||||||
res->filesystem_cache_name = ast->as<ASTLiteral>()->value.safeGet<String>();
|
res->filesystem_cache_name = ast->as<ASTLiteral>()->value.safeGet<String>();
|
||||||
|
if (ParserKeyword{"KEY"}.ignore(pos, expected) && ParserIdentifier().parse(pos, ast, expected))
|
||||||
|
{
|
||||||
|
res->key_to_drop = ast->as<ASTIdentifier>()->name();
|
||||||
|
if (ParserKeyword{"OFFSET"}.ignore(pos, expected) && ParserLiteral().parse(pos, ast, expected))
|
||||||
|
res->offset_to_drop = ast->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||||
|
}
|
||||||
|
}
|
||||||
if (!parseQueryWithOnCluster(res, pos, expected))
|
if (!parseQueryWithOnCluster(res, pos, expected))
|
||||||
return false;
|
return false;
|
||||||
break;
|
break;
|
||||||
|
@ -0,0 +1,5 @@
|
|||||||
|
OK
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
71
tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh
Executable file
71
tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh
Executable file
@ -0,0 +1,71 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings
|
||||||
|
|
||||||
|
# set -x
|
||||||
|
|
||||||
|
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CUR_DIR"/../shell_config.sh
|
||||||
|
|
||||||
|
|
||||||
|
disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}"
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
DROP TABLE IF EXISTS test;
|
||||||
|
CREATE TABLE test (a Int32, b String)
|
||||||
|
ENGINE = MergeTree() ORDER BY tuple()
|
||||||
|
SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk);
|
||||||
|
|
||||||
|
INSERT INTO test SELECT 1, 'test';
|
||||||
|
"""
|
||||||
|
|
||||||
|
query_id=$RANDOM
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1"
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY kek;
|
||||||
|
""" 2>&1 | grep -q "Invalid cache key hex: kek" && echo "OK" || echo "FAIL"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} -q " system flush logs"
|
||||||
|
|
||||||
|
key=$($CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1;
|
||||||
|
""")
|
||||||
|
|
||||||
|
offset=$($CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT offset FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1;
|
||||||
|
""")
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT count() FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset;
|
||||||
|
"""
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY $key OFFSET $offset;
|
||||||
|
"""
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT count() FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset;
|
||||||
|
"""
|
||||||
|
|
||||||
|
query_id=$RANDOM$RANDOM
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} -q " system flush logs"
|
||||||
|
|
||||||
|
key=$($CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1;
|
||||||
|
""")
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT count() FROM system.filesystem_cache WHERE key = '$key';
|
||||||
|
"""
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY $key
|
||||||
|
"""
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -nm --query """
|
||||||
|
SELECT count() FROM system.filesystem_cache WHERE key = '$key';
|
||||||
|
"""
|
Loading…
Reference in New Issue
Block a user