mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #11644 from ClickHouse/remove-trivial-count-optimization-for-row-level-security
Remove trivial count query optimization if row-level security is set
This commit is contained in:
commit
47316dd59d
7
programs/server/users.d/access_management.xml
Normal file
7
programs/server/users.d/access_management.xml
Normal file
@ -0,0 +1,7 @@
|
||||
<yandex>
|
||||
<users>
|
||||
<default>
|
||||
<access_management>1</access_management>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -208,7 +208,9 @@ struct ExpressionAnalysisResult
|
||||
const FilterInfoPtr & filter_info,
|
||||
const Block & source_header);
|
||||
|
||||
/// Filter for row-level security.
|
||||
bool hasFilter() const { return filter_info.get(); }
|
||||
|
||||
bool hasJoin() const { return before_join.get(); }
|
||||
bool hasPrewhere() const { return prewhere_info.get(); }
|
||||
bool hasWhere() const { return before_where.get(); }
|
||||
|
@ -94,7 +94,8 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
/// Assumes `storage` is set and the table filter (row-level security) is not empty.
|
||||
String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const
|
||||
String InterpreterSelectQuery::generateFilterActions(
|
||||
ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const
|
||||
{
|
||||
const auto & db_name = table_id.getDatabaseName();
|
||||
const auto & table_name = table_id.getTableName();
|
||||
@ -474,8 +475,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
second_stage,
|
||||
options.only_analyze,
|
||||
filter_info,
|
||||
source_header
|
||||
);
|
||||
source_header);
|
||||
|
||||
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
|
||||
{
|
||||
@ -979,10 +979,13 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
|
||||
/// Optimization for trivial query like SELECT count() FROM table.
|
||||
bool optimize_trivial_count =
|
||||
syntax_analyzer_result->optimize_trivial_count && storage &&
|
||||
processing_stage == QueryProcessingStage::FetchColumns &&
|
||||
query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) &&
|
||||
typeid_cast<AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
|
||||
syntax_analyzer_result->optimize_trivial_count
|
||||
&& storage
|
||||
&& !filter_info
|
||||
&& processing_stage == QueryProcessingStage::FetchColumns
|
||||
&& query_analyzer->hasAggregation()
|
||||
&& (query_analyzer->aggregates().size() == 1)
|
||||
&& typeid_cast<AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
|
||||
|
||||
if (optimize_trivial_count)
|
||||
{
|
||||
|
@ -132,7 +132,8 @@ private:
|
||||
void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
|
||||
void executeMergeSorted(QueryPipeline & pipeline, const SortDescription & sort_description, UInt64 limit);
|
||||
|
||||
String generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const;
|
||||
String generateFilterActions(
|
||||
ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const;
|
||||
|
||||
enum class Modificator
|
||||
{
|
||||
@ -159,6 +160,7 @@ private:
|
||||
|
||||
/// Is calculated in getSampleBlock. Is used later in readImpl.
|
||||
ExpressionAnalysisResult analysis_result;
|
||||
/// For row-level security.
|
||||
FilterInfoPtr filter_info;
|
||||
|
||||
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
|
||||
|
@ -198,7 +198,8 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id)
|
||||
void ReplicatedMergeTreeBlockOutputStream::commitPart(
|
||||
zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id)
|
||||
{
|
||||
storage.check(part->getColumns());
|
||||
assertSessionIsNotExpired(zookeeper);
|
||||
|
@ -0,0 +1,3 @@
|
||||
3
|
||||
2
|
||||
3
|
@ -0,0 +1,12 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
|
||||
CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||
INSERT INTO t VALUES (1), (2), (3);
|
||||
|
||||
SELECT count() FROM t;
|
||||
CREATE ROW POLICY filter ON t USING (x % 2 = 1) TO ALL;
|
||||
SELECT count() FROM t;
|
||||
DROP ROW POLICY filter ON t;
|
||||
SELECT count() FROM t;
|
||||
|
||||
DROP TABLE t;
|
Loading…
Reference in New Issue
Block a user