mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Fix performance regression of scalar query
This commit is contained in:
parent
54b4049ea5
commit
df06f9f974
@ -78,7 +78,7 @@ public:
|
||||
|
||||
static ColumnWithTypeAndName createScalar(ContextPtr context_)
|
||||
{
|
||||
if (const auto * block = context_->tryGetLocalScalar(Scalar::scalar_name))
|
||||
if (const auto * block = context_->tryGetSpecialScalar(Scalar::scalar_name))
|
||||
return block->getByPosition(0);
|
||||
else if (context_->hasQueryContext())
|
||||
{
|
||||
|
@ -930,10 +930,10 @@ const Block & Context::getScalar(const String & name) const
|
||||
return it->second;
|
||||
}
|
||||
|
||||
const Block * Context::tryGetLocalScalar(const String & name) const
|
||||
const Block * Context::tryGetSpecialScalar(const String & name) const
|
||||
{
|
||||
auto it = local_scalars.find(name);
|
||||
if (local_scalars.end() == it)
|
||||
auto it = special_scalars.find(name);
|
||||
if (special_scalars.end() == it)
|
||||
return nullptr;
|
||||
return &it->second;
|
||||
}
|
||||
@ -1004,12 +1004,12 @@ void Context::addScalar(const String & name, const Block & block)
|
||||
}
|
||||
|
||||
|
||||
void Context::addLocalScalar(const String & name, const Block & block)
|
||||
void Context::addSpecialScalar(const String & name, const Block & block)
|
||||
{
|
||||
if (isGlobalContext())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have local scalars");
|
||||
|
||||
local_scalars[name] = block;
|
||||
special_scalars[name] = block;
|
||||
}
|
||||
|
||||
|
||||
|
@ -227,7 +227,8 @@ private:
|
||||
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
|
||||
TemporaryTablesMapping external_tables_mapping;
|
||||
Scalars scalars;
|
||||
Scalars local_scalars;
|
||||
/// Used to store constant values which are different on each instance during distributed plan, such as _shard_num.
|
||||
Scalars special_scalars;
|
||||
|
||||
/// Used in s3Cluster table function. With this callback, a worker node could ask an initiator
|
||||
/// about next file to read from s3.
|
||||
@ -487,8 +488,8 @@ public:
|
||||
void addScalar(const String & name, const Block & block);
|
||||
bool hasScalar(const String & name) const;
|
||||
|
||||
const Block * tryGetLocalScalar(const String & name) const;
|
||||
void addLocalScalar(const String & name, const Block & block);
|
||||
const Block * tryGetSpecialScalar(const String & name) const;
|
||||
void addSpecialScalar(const String & name, const Block & block);
|
||||
|
||||
const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; }
|
||||
void addQueryAccessInfo(
|
||||
|
@ -113,14 +113,18 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
bool is_local = false;
|
||||
|
||||
Block scalar;
|
||||
if (data.local_scalars.count(scalar_query_hash_str))
|
||||
if (data.only_analyze)
|
||||
{
|
||||
/// Don't use scalar cache during query analysis
|
||||
}
|
||||
else if (data.local_scalars.contains(scalar_query_hash_str))
|
||||
{
|
||||
hit = true;
|
||||
scalar = data.local_scalars[scalar_query_hash_str];
|
||||
is_local = true;
|
||||
ProfileEvents::increment(ProfileEvents::ScalarSubqueriesLocalCacheHit);
|
||||
}
|
||||
else if (data.scalars.count(scalar_query_hash_str))
|
||||
else if (data.scalars.contains(scalar_query_hash_str))
|
||||
{
|
||||
hit = true;
|
||||
scalar = data.scalars[scalar_query_hash_str];
|
||||
|
@ -18,11 +18,11 @@ public:
|
||||
, max_streams(context->getSettingsRef().max_threads)
|
||||
{
|
||||
if (options.shard_num)
|
||||
context->addLocalScalar(
|
||||
context->addSpecialScalar(
|
||||
"_shard_num",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_num), std::make_shared<DataTypeUInt32>(), "_shard_num"}});
|
||||
if (options.shard_count)
|
||||
context->addLocalScalar(
|
||||
context->addSpecialScalar(
|
||||
"_shard_count",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared<DataTypeUInt32>(), "_shard_count"}});
|
||||
}
|
||||
|
17
tests/performance/scalar2.xml
Normal file
17
tests/performance/scalar2.xml
Normal file
@ -0,0 +1,17 @@
|
||||
<test>
|
||||
<create_query>CREATE TABLE tbl0 (`ds` Date, `x1` String, `x2` UInt32, `x3` UInt32, `x4` UInt32, `bm` AggregateFunction(groupBitmap, UInt32)) ENGINE = MergeTree PARTITION BY (ds, x1) ORDER BY (x2, x3, x4) SETTINGS index_granularity = 1</create_query>
|
||||
|
||||
<create_query>CREATE TABLE tbl (`ds` Date, `y1` UInt32, `x4` UInt32, `y2` UInt32, `y3` UInt32, `bm` AggregateFunction(groupBitmap, UInt32), `y4` UInt32 DEFAULT 0) ENGINE = MergeTree PARTITION BY (ds) ORDER BY (x4, y2, y3) SETTINGS index_granularity = 8192, max_parts_in_total = 10000000</create_query>
|
||||
|
||||
<fill_query>insert into tbl0 with murmurHash3_32(toUInt32(rand())) as uid select toDate('2022-03-01')+rand()%7 as ds, concat('xx',toString(rand()%10+1)) as x1, 1 as x2, 2 as x3, bitShiftRight(uid, 22) as x4, groupBitmapState(uid) as bm from numbers(100000000) where x4%40=0 group by ds, x1, x2, x3, x4</fill_query>
|
||||
|
||||
<fill_query>insert into tbl with murmurHash3_32(toUInt32(rand())) as uid select toDate('2022-03-01')+rand()%7 as ds, rand()%1000+5000 as y1, bitShiftRight(uid, 22) as x4, rand()%100 as y2, rand()%2000 as y3, groupBitmapState(uid) as bm, rand()%1 as y4 from numbers(100000000) where x4%40=0 group by ds, y1, x4, y2, y3, y4</fill_query>
|
||||
|
||||
<create_query>CREATE TABLE tmp_acc_hit engine Memory AS SELECT x1, x2, x3, arrayReduceInRanges('groupBitmapMergeState', [(1, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6), (1, 7)], bs) AS bs FROM (SELECT x1, x2, x3, groupArrayInsertAt(b, multiIf(ds = '2022-03-01', 0, ds = '2022-03-02', 1, ds = '2022-03-03', 2, ds = '2022-03-04', 3, ds = '2022-03-05', 4, ds = '2022-03-06', 5, ds = '2022-03-07', 6, 7)) AS bs FROM (SELECT x1, x2, x3, ds, groupBitmapOrState(bm) AS b FROM tbl0 WHERE ((ds >= '2022-03-01') AND (ds <= '2022-03-07')) AND (((x1 = 'xx1') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx2') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx3') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx4') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx5') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx6') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx7') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx8') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx9') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx10') AND (x2 = 1) AND (x3 = 2))) AND (x4 IN (0, 40, 80, 120, 160, 200, 240, 280, 320, 360, 400, 440, 480, 520, 560, 600, 640, 680, 720, 760, 800, 840, 880, 920, 960, 1000)) GROUP BY x1, x2, x3, ds) AS t_hit GROUP BY x1, x2, x3)</create_query>
|
||||
|
||||
<query>WITH (SELECT groupArrayInsertAt(b, multiIf((x1 = 'xx1') AND (x2 = 1) AND (x3 = 2), 0, (x1 = 'xx2') AND (x2 = 1) AND (x3 = 2), 1, (x1 = 'xx3') AND (x2 = 1) AND (x3 = 2), 2, (x1 = 'xx4') AND (x2 = 1) AND (x3 = 2), 3, (x1 = 'xx5') AND (x2 = 1) AND (x3 = 2), 4, (x1 = 'xx6') AND (x2 = 1) AND (x3 = 2), 5, (x1 = 'xx7') AND (x2 = 1) AND (x3 = 2), 6, (x1 = 'xx8') AND (x2 = 1) AND (x3 = 2), 7, (x1 = 'xx9') AND (x2 = 1) AND (x3 = 2), 8, (x1 = 'xx10') AND (x2 = 1) AND (x3 = 2), 9, 10)) FROM (SELECT x1, x2, x3, bs AS b FROM tmp_acc_hit)) AS bs SELECT y1, x4, toString(flat_arr) AS flat_arr, toString([bitmapAndCardinality(bmor1, (bs[1])[1]), bitmapAndCardinality(bmor2, (bs[1])[1]), bitmapAndCardinality(bmor3, (bs[1])[1]), bitmapAndCardinality(bmor1, (bs[2])[1]), bitmapAndCardinality(bmor2, (bs[2])[1]), bitmapAndCardinality(bmor3, (bs[2])[1]), bitmapAndCardinality(bmor1, (bs[3])[1]), bitmapAndCardinality(bmor2, (bs[3])[1]), bitmapAndCardinality(bmor3, (bs[3])[1]), bitmapAndCardinality(bmor1, (bs[4])[1]), bitmapAndCardinality(bmor2, (bs[4])[1]), bitmapAndCardinality(bmor3, (bs[4])[1]), bitmapAndCardinality(bmor1, (bs[5])[1]), bitmapAndCardinality(bmor2, (bs[5])[1]), bitmapAndCardinality(bmor3, (bs[5])[1]), bitmapAndCardinality(bmor1, (bs[6])[1]), bitmapAndCardinality(bmor2, (bs[6])[1]), bitmapAndCardinality(bmor3, (bs[6])[1]), bitmapAndCardinality(bmor1, (bs[7])[1]), bitmapAndCardinality(bmor2, (bs[7])[1]), bitmapAndCardinality(bmor3, (bs[7])[1]), bitmapAndCardinality(bmor1, (bs[8])[1]), bitmapAndCardinality(bmor2, (bs[8])[1]), bitmapAndCardinality(bmor3, (bs[8])[1]), bitmapAndCardinality(bmor1, (bs[9])[1]), bitmapAndCardinality(bmor2, (bs[9])[1]), bitmapAndCardinality(bmor3, (bs[9])[1]), bitmapAndCardinality(bmor1, (bs[10])[1]), bitmapAndCardinality(bmor2, (bs[10])[1]), bitmapAndCardinality(bmor3, (bs[10])[1])]) AS flat_arr_2 from (SELECT toString(y1) AS y1, toString(x4) AS x4, arrayFlatten(groupArrayInsertAt(flat_arr, multiIf(date_ = '2022-03-01', 0, 1))) AS flat_arr, groupBitmapOrState(bmor1) AS bmor1, groupBitmapOrState(bmor2) AS bmor2, groupBitmapOrState(bmor3) AS bmor3 FROM (WITH '2022-03-01' AS start_ds SELECT y1, x4, groupBitmapOrState(bm) AS bmor1, groupBitmapOrStateIf(bm, y2 > 0) AS bmor2, groupBitmapOrStateIf(bm, y4 = 1) AS bmor3, [sum(y2 * bitmapAndCardinality(bm, (bs[1])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[2])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[3])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[4])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[5])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[6])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[7])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[8])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[9])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[10])[1]))] AS flat_arr, start_ds AS date_ FROM tbl WHERE (ds = start_ds) AND (y1 IN (7063, 5010, 5006, 6788, 6176, 6203, 6769, 6555, 7062, 5119, 5007, 5212, 6814, 6177, 6789, 5095, 4942, 6243, 7061, 6744, 6201, 7196, 6181, 7195, 6178, 5004, 6790, 5008, 6877, 7281, 6791, 6179, 5214, 5005, 7146, 6980, 6322, 5222, 5217, 5137, 6561, 5133, 6937, 5142, 5130, 6885, 7250, 5103, 6867, 7066, 5096, 6868, 6199, 7269, 5131, 6414, 6884, 6560, 5136, 6883, 5158, 6869, 5097, 5132, 5102, 7251, 5219, 4695, 5220, 5202, 4203, 4204, 5098, 6870, 7064, 5101, 5105, 5140, 5135, 5139, 6880, 6194, 5218, 4202, 6655, 5104, 5183, 7245, 5100, 7065, 5099, 6938, 5138, 6881, 5134, 6886, 5141, 5129)) AND (x4 IN (0, 40, 80, 120, 160, 200, 240, 280, 320, 360, 400, 440, 480, 520, 560, 600, 640, 680, 720, 760, 800, 840, 880, 920, 960, 1000)) AND (y4 IN (0, 1)) GROUP BY y1, x4) GROUP BY y1, x4) LIMIT 1</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS tbl</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS tbl0</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS tmp_acc_hit</drop_query>
|
||||
</test>
|
Loading…
Reference in New Issue
Block a user