mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
fix assertion when filtering tables in StorageMerge
This commit is contained in:
parent
4f34bc4e73
commit
ad414bab05
@ -159,8 +159,7 @@ bool StorageMerge::isRemote() const
|
||||
bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & /*metadata_snapshot*/) const
|
||||
{
|
||||
/// It's beneficial if it is true for at least one table.
|
||||
StorageListWithLocks selected_tables = getSelectedTables(
|
||||
query_context->getCurrentQueryId(), query_context->getSettingsRef());
|
||||
StorageListWithLocks selected_tables = getSelectedTables(query_context);
|
||||
|
||||
size_t i = 0;
|
||||
for (const auto & table : selected_tables)
|
||||
@ -250,8 +249,7 @@ Pipe StorageMerge::read(
|
||||
/** First we make list of selected tables to find out its size.
|
||||
* This is necessary to correctly pass the recommended number of threads to each table.
|
||||
*/
|
||||
StorageListWithLocks selected_tables
|
||||
= getSelectedTables(query_info, has_table_virtual_column, local_context->getCurrentQueryId(), local_context->getSettingsRef());
|
||||
StorageListWithLocks selected_tables = getSelectedTables(local_context, query_info.query, has_table_virtual_column);
|
||||
|
||||
if (selected_tables.empty())
|
||||
/// FIXME: do we support sampling in this case?
|
||||
@ -427,34 +425,20 @@ Pipe StorageMerge::createSources(
|
||||
return pipe;
|
||||
}
|
||||
|
||||
|
||||
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String & query_id, const Settings & settings) const
|
||||
{
|
||||
StorageListWithLocks selected_tables;
|
||||
auto iterator = getDatabaseIterator(getContext());
|
||||
|
||||
while (iterator->isValid())
|
||||
{
|
||||
const auto & table = iterator->table();
|
||||
if (table && table.get() != this)
|
||||
selected_tables.emplace_back(
|
||||
table, table->lockForShare(query_id, settings.lock_acquire_timeout), iterator->name());
|
||||
|
||||
iterator->next();
|
||||
}
|
||||
|
||||
return selected_tables;
|
||||
}
|
||||
|
||||
|
||||
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
|
||||
const SelectQueryInfo & query_info, bool has_virtual_column, const String & query_id, const Settings & settings) const
|
||||
ContextPtr query_context,
|
||||
const ASTPtr & query /* = nullptr */,
|
||||
bool filter_by_virtual_column /* = false */) const
|
||||
{
|
||||
const ASTPtr & query = query_info.query;
|
||||
assert(!filter_by_virtual_column || query);
|
||||
|
||||
const Settings & settings = query_context->getSettingsRef();
|
||||
StorageListWithLocks selected_tables;
|
||||
DatabaseTablesIteratorPtr iterator = getDatabaseIterator(getContext());
|
||||
|
||||
auto virtual_column = ColumnString::create();
|
||||
MutableColumnPtr table_name_virtual_column;
|
||||
if (filter_by_virtual_column)
|
||||
table_name_virtual_column = ColumnString::create();
|
||||
|
||||
while (iterator->isValid())
|
||||
{
|
||||
@ -467,18 +451,20 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
|
||||
|
||||
if (storage.get() != this)
|
||||
{
|
||||
selected_tables.emplace_back(
|
||||
storage, storage->lockForShare(query_id, settings.lock_acquire_timeout), iterator->name());
|
||||
virtual_column->insert(iterator->name());
|
||||
auto table_lock = storage->lockForShare(query_context->getCurrentQueryId(), settings.lock_acquire_timeout);
|
||||
selected_tables.emplace_back(storage, std::move(table_lock), iterator->name());
|
||||
if (filter_by_virtual_column)
|
||||
table_name_virtual_column->insert(iterator->name());
|
||||
}
|
||||
|
||||
iterator->next();
|
||||
}
|
||||
|
||||
if (has_virtual_column)
|
||||
if (filter_by_virtual_column)
|
||||
{
|
||||
Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(virtual_column), std::make_shared<DataTypeString>(), "_table")};
|
||||
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, getContext());
|
||||
/// Filter names of selected tables if there is a condition on "_table" virtual column in WHERE clause
|
||||
Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(table_name_virtual_column), std::make_shared<DataTypeString>(), "_table")};
|
||||
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, query_context);
|
||||
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
|
||||
|
||||
/// Remove unused tables from the list
|
||||
@ -488,7 +474,6 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
|
||||
return selected_tables;
|
||||
}
|
||||
|
||||
|
||||
DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(ContextPtr local_context) const
|
||||
{
|
||||
try
|
||||
|
@ -55,10 +55,8 @@ private:
|
||||
using StorageWithLockAndName = std::tuple<StoragePtr, TableLockHolder, String>;
|
||||
using StorageListWithLocks = std::list<StorageWithLockAndName>;
|
||||
|
||||
StorageListWithLocks getSelectedTables(const String & query_id, const Settings & settings) const;
|
||||
|
||||
StorageMerge::StorageListWithLocks getSelectedTables(
|
||||
const SelectQueryInfo & query_info, bool has_virtual_column, const String & query_id, const Settings & settings) const;
|
||||
ContextPtr query_context, const ASTPtr & query = nullptr, bool filter_by_virtual_column = false) const;
|
||||
|
||||
template <typename F>
|
||||
StoragePtr getFirstTable(F && predicate) const;
|
||||
|
@ -0,0 +1,7 @@
|
||||
30 4995
|
||||
20 4950
|
||||
15 4700
|
||||
20 495
|
||||
20 4545
|
||||
15 470
|
||||
15 4520
|
@ -0,0 +1,26 @@
|
||||
drop table if exists src_table_1;
|
||||
drop table if exists src_table_2;
|
||||
drop table if exists src_table_3;
|
||||
drop table if exists set;
|
||||
|
||||
create table src_table_1 (n UInt64) engine=Memory as select * from numbers(10);
|
||||
create table src_table_2 (n UInt64) engine=Log as select number * 10 from numbers(10);
|
||||
create table src_table_3 (n UInt64) engine=MergeTree order by n as select number * 100 from numbers(10);
|
||||
create table set (s String) engine=Set as select arrayJoin(['src_table_1', 'src_table_2']);
|
||||
|
||||
create temporary table tmp (s String);
|
||||
insert into tmp values ('src_table_1'), ('src_table_3');
|
||||
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table');
|
||||
-- FIXME #21401 select count(), sum(n) from merge(currentDatabase(), 'src_table') where _table = 'src_table_1' or toInt8(substr(_table, 11, 1)) = 2;
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table') where _table in ('src_table_2', 'src_table_3');
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table') where _table in ('src_table_2', 'src_table_3') and n % 20 = 0;
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table') where _table in set;
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table') where _table in tmp;
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table') where _table in set and n % 2 = 0;
|
||||
select count(), sum(n) from merge(currentDatabase(), 'src_table') where n % 2 = 0 and _table in tmp;
|
||||
|
||||
drop table src_table_1;
|
||||
drop table src_table_2;
|
||||
drop table src_table_3;
|
||||
drop table set;
|
Loading…
Reference in New Issue
Block a user