Skip to content

Commit

Permalink
Backport #59351 to 23.12: Fix not-ready set for system.tables
Browse files Browse the repository at this point in the history
  • Loading branch information
robot-clickhouse committed Jan 30, 2024
1 parent 608407c commit 22cb900
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 3 deletions.
14 changes: 11 additions & 3 deletions src/Storages/System/StorageSystemTables.cpp
Expand Up @@ -664,10 +664,15 @@ class ReadFromSystemTables : public SourceStepWithFilter
{
}

void applyFilters() override;

private:
ContextPtr context;
std::vector<UInt8> columns_mask;
size_t max_block_size;

ColumnPtr filtered_databases_column;
ColumnPtr filtered_tables_column;
};

void StorageSystemTables::read(
Expand All @@ -694,16 +699,19 @@ void StorageSystemTables::read(
query_plan.addStep(std::move(reading));
}

void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
void ReadFromSystemTables::applyFilters()
{
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
const ActionsDAG::Node * predicate = nullptr;
if (filter_actions_dag)
predicate = filter_actions_dag->getOutputs().at(0);

ColumnPtr filtered_databases_column = getFilteredDatabases(predicate, context);
ColumnPtr filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context);
filtered_databases_column = getFilteredDatabases(predicate, context);
filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context);
}

void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
Pipe pipe(std::make_shared<TablesBlockSource>(
std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases_column), std::move(filtered_tables_column), context));
pipeline.init(std::move(pipe));
Expand Down
1 change: 1 addition & 0 deletions tests/queries/0_stateless/02841_not_ready_set_bug.sh
Expand Up @@ -9,3 +9,4 @@ $CLICKHOUSE_CLIENT -q "create table t1 (number UInt64) engine = MergeTree order
$CLICKHOUSE_CLIENT -q "insert into t1 select number from numbers(10);"
$CLICKHOUSE_CLIENT --max_threads=2 --max_result_rows=1 --result_overflow_mode=break -q "with tab as (select min(number) from t1 prewhere number in (select number from view(select number, row_number() OVER (partition by number % 2 ORDER BY number DESC) from numbers_mt(1e4)) where number != 2 order by number)) select number from t1 union all select * from tab;" > /dev/null

$CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE 1 in (SELECT number from numbers(2)) AND database = currentDatabase() format Null"

0 comments on commit 22cb900

Please sign in to comment.