Skip to content

Commit

Permalink
Merge pull request #55621 from ClickHouse/merge-52352
Browse files Browse the repository at this point in the history
Merging #52352
  • Loading branch information
davenger committed Oct 17, 2023
2 parents 9629e25 + 395a549 commit 11a62bc
Show file tree
Hide file tree
Showing 4 changed files with 108 additions and 9 deletions.
69 changes: 62 additions & 7 deletions src/Storages/StorageSet.cpp
Expand Up @@ -156,12 +156,62 @@ StorageSet::StorageSet(
}


void StorageSet::insertBlock(const Block & block, ContextPtr) { set->insertFromBlock(block.getColumnsWithTypeAndName()); }
void StorageSet::finishInsert() { set->finishInsert(); }
SetPtr StorageSet::getSet() const
{
std::lock_guard lock(mutex);
return set;
}


void StorageSet::insertBlock(const Block & block, ContextPtr)
{
SetPtr current_set;
{
std::lock_guard lock(mutex);
current_set = set;
}
current_set->insertFromBlock(block.getColumnsWithTypeAndName());
}

size_t StorageSet::getSize(ContextPtr) const { return set->getTotalRowCount(); }
std::optional<UInt64> StorageSet::totalRows(const Settings &) const { return set->getTotalRowCount(); }
std::optional<UInt64> StorageSet::totalBytes(const Settings &) const { return set->getTotalByteCount(); }
void StorageSet::finishInsert()
{
SetPtr current_set;
{
std::lock_guard lock(mutex);
current_set = set;
}
current_set->finishInsert();
}

size_t StorageSet::getSize(ContextPtr) const
{
SetPtr current_set;
{
std::lock_guard lock(mutex);
current_set = set;
}
return current_set->getTotalRowCount();
}

std::optional<UInt64> StorageSet::totalRows(const Settings &) const
{
SetPtr current_set;
{
std::lock_guard lock(mutex);
current_set = set;
}
return current_set->getTotalRowCount();
}

std::optional<UInt64> StorageSet::totalBytes(const Settings &) const
{
SetPtr current_set;
{
std::lock_guard lock(mutex);
current_set = set;
}
return current_set->getTotalByteCount();
}

void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &)
{
Expand All @@ -176,8 +226,13 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn
Block header = metadata_snapshot->getSampleBlock();

increment = 0;
set = std::make_shared<Set>(SizeLimits(), 0, true);
set->setHeader(header.getColumnsWithTypeAndName());

auto new_set = std::make_shared<Set>(SizeLimits(), 0, true);
new_set->setHeader(header.getColumnsWithTypeAndName());
{
std::lock_guard lock(mutex);
set = new_set;
}
}


Expand Down
6 changes: 4 additions & 2 deletions src/Storages/StorageSet.h
Expand Up @@ -79,15 +79,17 @@ class StorageSet final : public StorageSetOrJoinBase
String getName() const override { return "Set"; }

/// Access the insides.
SetPtr & getSet() { return set; }
SetPtr getSet() const;

void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override;

std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;

private:
SetPtr set;
/// Allows to concurrently truncate the set and work (read/fill) the existing set.
mutable std::mutex mutex;
SetPtr set TSA_GUARDED_BY(mutex);

void insertBlock(const Block & block, ContextPtr) override;
void finishInsert() override;
Expand Down
Empty file.
42 changes: 42 additions & 0 deletions tests/queries/0_stateless/02867_storage_set_tsan.sh
@@ -0,0 +1,42 @@
#!/usr/bin/env bash
# Tags: race, no-debug

CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh

$CLICKHOUSE_CLIENT -mn -q """
DROP TABLE IF EXISTS t1_02867;
CREATE TABLE t1_02867 (x UInt64) ENGINE=Set();
"""

function repeat_select() {
n=0
while [ "$n" -lt 20 ];
do
n=$(( n + 1 ))
$CLICKHOUSE_CLIENT -q "SELECT count() as a FROM numbers(10) WHERE number IN t1_02867" > /dev/null 2> /dev/null || exit
done
}

function repeat_truncate_insert() {
n=0
while [ "$n" -lt 20 ];
do
n=$(( n + 1 ))
$CLICKHOUSE_CLIENT -q "TRUNCATE t1_02867;" > /dev/null 2> /dev/null || exit
done
}

repeat_select &
repeat_truncate_insert &
repeat_select &
repeat_truncate_insert &
repeat_select &
repeat_truncate_insert &
repeat_select &
repeat_truncate_insert &

sleep 10

$CLICKHOUSE_CLIENT -mn -q "DROP TABLE IF EXISTS t1_02867;"

0 comments on commit 11a62bc

Please sign in to comment.