Skip to content

Commit

Permalink
Merge pull request ClickHouse#60139 from ClickHouse/revert-60093-lazy…
Browse files Browse the repository at this point in the history
…-primary-key-loading

Fix data race in `IMergeTreeDataPart`
  • Loading branch information
alexey-milovidov authored Feb 20, 2024
2 parents 0bd3016 + 7c2654f commit b216223
Show file tree
Hide file tree
Showing 2 changed files with 8 additions and 5 deletions.
6 changes: 4 additions & 2 deletions src/Storages/MergeTree/IMergeTreeDataPart.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -347,7 +347,7 @@ const IMergeTreeDataPart::Index & IMergeTreeDataPart::getIndex() const
{
std::scoped_lock lock(index_mutex);
if (!index_loaded)
loadIndex(lock);
loadIndex();
index_loaded = true;
return index;
}
Expand Down Expand Up @@ -569,6 +569,7 @@ void IMergeTreeDataPart::removeIfNeeded()

UInt64 IMergeTreeDataPart::getIndexSizeInBytes() const
{
std::scoped_lock lock(index_mutex);
UInt64 res = 0;
for (const ColumnPtr & column : index)
res += column->byteSize();
Expand All @@ -577,6 +578,7 @@ UInt64 IMergeTreeDataPart::getIndexSizeInBytes() const

UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const
{
std::scoped_lock lock(index_mutex);
UInt64 res = 0;
for (const ColumnPtr & column : index)
res += column->allocatedBytes();
Expand Down Expand Up @@ -828,7 +830,7 @@ void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) co
{
}

void IMergeTreeDataPart::loadIndex(std::scoped_lock<std::mutex> &) const
void IMergeTreeDataPart::loadIndex() const
{
/// Memory for index must not be accounted as memory usage for query, because it belongs to a table.
MemoryTrackerBlockerInThread temporarily_disable_memory_tracker;
Expand Down
7 changes: 4 additions & 3 deletions src/Storages/MergeTree/IMergeTreeDataPart.h
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
#include <IO/WriteSettings.h>
#include <Core/Block.h>
#include <base/types.h>
#include <base/defines.h>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage.h>
#include <Storages/LightweightDeleteDescription.h>
Expand Down Expand Up @@ -565,8 +566,8 @@ class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPar
/// Lazily loaded in RAM. Contains each index_granularity-th value of primary key tuple.
/// Note that marks (also correspond to primary key) are not always in RAM, but cached. See MarkCache.h.
mutable std::mutex index_mutex;
mutable Index index;
mutable bool index_loaded = false;
mutable Index index TSA_GUARDED_BY(index_mutex);
mutable bool index_loaded TSA_GUARDED_BY(index_mutex) = false;

/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk
ColumnSize total_columns_size;
Expand Down Expand Up @@ -664,7 +665,7 @@ class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPar
virtual void appendFilesOfIndexGranularity(Strings & files) const;

/// Loads the index file.
void loadIndex(std::scoped_lock<std::mutex> &) const;
void loadIndex() const TSA_REQUIRES(index_mutex);

void appendFilesOfIndex(Strings & files) const;

Expand Down

0 comments on commit b216223

Please sign in to comment.