mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #29355 from ClickHouse/remove-trash-db-regexp
Fix assert in table function `merge` with database regexp
This commit is contained in:
commit
5a7f30171a
@ -162,7 +162,7 @@ IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const
|
||||
{
|
||||
/// database.table.column
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table))
|
||||
return ColumnMatch::DbAndTable;
|
||||
return ColumnMatch::DBAndTable;
|
||||
|
||||
/// alias.column
|
||||
if (doesIdentifierBelongTo(identifier, db_and_table.alias))
|
||||
@ -199,7 +199,7 @@ void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const Da
|
||||
case ColumnMatch::TableAlias:
|
||||
to_strip = 1;
|
||||
break;
|
||||
case ColumnMatch::DbAndTable:
|
||||
case ColumnMatch::DBAndTable:
|
||||
to_strip = 2;
|
||||
break;
|
||||
default:
|
||||
|
@ -31,7 +31,7 @@ struct IdentifierSemantic
|
||||
ColumnName, /// column qualified with column names list
|
||||
AliasedTableName, /// column qualified with table name (but table has an alias so its priority is lower than TableName)
|
||||
TableName, /// column qualified with table name
|
||||
DbAndTable, /// column qualified with database and table name
|
||||
DBAndTable, /// column qualified with database and table name
|
||||
TableAlias, /// column qualified with table alias
|
||||
Ambiguous,
|
||||
};
|
||||
|
@ -128,7 +128,7 @@ private:
|
||||
/// Table has an alias. We do not need to rewrite qualified names with table alias (match == ColumnMatch::TableName).
|
||||
auto match = IdentifierSemantic::canReferColumnToTable(identifier, table);
|
||||
if (match == IdentifierSemantic::ColumnMatch::AliasedTableName ||
|
||||
match == IdentifierSemantic::ColumnMatch::DbAndTable)
|
||||
match == IdentifierSemantic::ColumnMatch::DBAndTable)
|
||||
{
|
||||
if (rewritten)
|
||||
throw Exception("Failed to rewrite distributed table names. Ambiguous column '" + identifier.name() + "'",
|
||||
|
@ -334,7 +334,7 @@ void RestoreQualifiedNamesMatcher::Data::changeTable(ASTIdentifier & identifier)
|
||||
{
|
||||
case IdentifierSemantic::ColumnMatch::AliasedTableName:
|
||||
case IdentifierSemantic::ColumnMatch::TableName:
|
||||
case IdentifierSemantic::ColumnMatch::DbAndTable:
|
||||
case IdentifierSemantic::ColumnMatch::DBAndTable:
|
||||
IdentifierSemantic::setColumnLongName(identifier, remote_table);
|
||||
break;
|
||||
default:
|
||||
|
@ -104,24 +104,6 @@ ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr
|
||||
return res;
|
||||
}
|
||||
|
||||
std::tuple<bool, ASTPtr> evaluateDatabaseNameForMergeEngine(const ASTPtr & node, ContextPtr context)
|
||||
{
|
||||
if (const auto * func = node->as<ASTFunction>(); func && func->name == "REGEXP")
|
||||
{
|
||||
if (func->arguments->children.size() != 1)
|
||||
throw Exception("Arguments for REGEXP in Merge ENGINE should be 1", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto * literal = func->arguments->children[0]->as<ASTLiteral>();
|
||||
if (!literal || literal->value.safeGet<String>().empty())
|
||||
throw Exception("Argument for REGEXP in Merge ENGINE should be a non empty String Literal", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return std::tuple{true, func->arguments->children[0]};
|
||||
}
|
||||
|
||||
auto ast = evaluateConstantExpressionForDatabaseName(node, context);
|
||||
return std::tuple{false, ast};
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
@ -53,6 +53,4 @@ ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr
|
||||
*/
|
||||
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr, size_t & limit);
|
||||
|
||||
// Evaluate database name or regexp for StorageMerge and TableFunction merge
|
||||
std::tuple<bool, ASTPtr> evaluateDatabaseNameForMergeEngine(const ASTPtr & node, ContextPtr context);
|
||||
}
|
||||
|
@ -280,7 +280,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
|
||||
{
|
||||
fs::create_directories(rocksdb_dir);
|
||||
}
|
||||
initDb();
|
||||
initDB();
|
||||
}
|
||||
|
||||
void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &)
|
||||
@ -291,10 +291,10 @@ void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr &
|
||||
|
||||
fs::remove_all(rocksdb_dir);
|
||||
fs::create_directories(rocksdb_dir);
|
||||
initDb();
|
||||
initDB();
|
||||
}
|
||||
|
||||
void StorageEmbeddedRocksDB::initDb()
|
||||
void StorageEmbeddedRocksDB::initDB()
|
||||
{
|
||||
rocksdb::Status status;
|
||||
rocksdb::Options base;
|
||||
|
@ -69,6 +69,6 @@ private:
|
||||
mutable std::shared_mutex rocksdb_ptr_mx;
|
||||
String rocksdb_dir;
|
||||
|
||||
void initDb();
|
||||
void initDB();
|
||||
};
|
||||
}
|
||||
|
@ -35,6 +35,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int ILLEGAL_PREWHERE;
|
||||
@ -49,7 +50,7 @@ StorageMerge::StorageMerge(
|
||||
const String & comment,
|
||||
const String & source_database_name_or_regexp_,
|
||||
bool database_is_regexp_,
|
||||
const DbToTableSetMap & source_databases_and_tables_,
|
||||
const DBToTableSetMap & source_databases_and_tables_,
|
||||
ContextPtr context_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
@ -573,11 +574,14 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const String & datab
|
||||
{
|
||||
auto database = DatabaseCatalog::instance().getDatabase(database_name);
|
||||
|
||||
auto table_name_match = [this, &database_name](const String & table_name_) -> bool {
|
||||
auto table_name_match = [this, database_name](const String & table_name_) -> bool
|
||||
{
|
||||
if (source_databases_and_tables)
|
||||
{
|
||||
const auto & source_tables = (*source_databases_and_tables).at(database_name);
|
||||
return source_tables.count(table_name_);
|
||||
if (auto it = source_databases_and_tables->find(database_name); it != source_databases_and_tables->end())
|
||||
return it->second.count(table_name_);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return source_table_regexp->match(table_name_);
|
||||
@ -742,6 +746,26 @@ IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const
|
||||
return first_materialized_mysql->getColumnSizes();
|
||||
}
|
||||
|
||||
|
||||
std::tuple<bool /* is_regexp */, ASTPtr> StorageMerge::evaluateDatabaseName(const ASTPtr & node, ContextPtr context_)
|
||||
{
|
||||
if (const auto * func = node->as<ASTFunction>(); func && func->name == "REGEXP")
|
||||
{
|
||||
if (func->arguments->children.size() != 1)
|
||||
throw Exception("REGEXP in Merge ENGINE takes only one argument", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto * literal = func->arguments->children[0]->as<ASTLiteral>();
|
||||
if (!literal || literal->value.safeGet<String>().empty())
|
||||
throw Exception("Argument for REGEXP in Merge ENGINE should be a non empty String Literal", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return {true, func->arguments->children[0]};
|
||||
}
|
||||
|
||||
auto ast = evaluateConstantExpressionForDatabaseName(node, context_);
|
||||
return {false, ast};
|
||||
}
|
||||
|
||||
|
||||
void registerStorageMerge(StorageFactory & factory)
|
||||
{
|
||||
factory.registerStorage("Merge", [](const StorageFactory::Arguments & args)
|
||||
@ -757,10 +781,11 @@ void registerStorageMerge(StorageFactory & factory)
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
auto [is_regexp, database_ast] = evaluateDatabaseNameForMergeEngine(engine_args[0], args.getLocalContext());
|
||||
auto [is_regexp, database_ast] = StorageMerge::evaluateDatabaseName(engine_args[0], args.getLocalContext());
|
||||
|
||||
if (!is_regexp)
|
||||
engine_args[0] = database_ast;
|
||||
|
||||
String source_database_name_or_regexp = database_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.getLocalContext());
|
||||
|
@ -48,12 +48,15 @@ public:
|
||||
bool mayBenefitFromIndexForIn(
|
||||
const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot) const override;
|
||||
|
||||
/// Evaluate database name or regexp for StorageMerge and TableFunction merge
|
||||
static std::tuple<bool /* is_regexp */, ASTPtr> evaluateDatabaseName(const ASTPtr & node, ContextPtr context);
|
||||
|
||||
private:
|
||||
using DbToTableSetMap = std::map<String, std::set<String>>;
|
||||
using DBToTableSetMap = std::map<String, std::set<String>>;
|
||||
|
||||
std::optional<OptimizedRegularExpression> source_database_regexp;
|
||||
std::optional<OptimizedRegularExpression> source_table_regexp;
|
||||
std::optional<DbToTableSetMap> source_databases_and_tables;
|
||||
std::optional<DBToTableSetMap> source_databases_and_tables;
|
||||
|
||||
String source_database_name_or_regexp;
|
||||
bool database_is_regexp = false;
|
||||
@ -86,7 +89,7 @@ protected:
|
||||
const String & comment,
|
||||
const String & source_database_name_or_regexp_,
|
||||
bool database_is_regexp_,
|
||||
const DbToTableSetMap & source_databases_and_tables_,
|
||||
const DBToTableSetMap & source_databases_and_tables_,
|
||||
ContextPtr context_);
|
||||
|
||||
StorageMerge(
|
||||
|
@ -52,7 +52,7 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
auto [is_regexp, database_ast] = evaluateDatabaseNameForMergeEngine(args[0], context);
|
||||
auto [is_regexp, database_ast] = StorageMerge::evaluateDatabaseName(args[0], context);
|
||||
|
||||
database_is_regexp = is_regexp;
|
||||
|
||||
@ -65,7 +65,7 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
}
|
||||
|
||||
|
||||
const TableFunctionMerge::DbToTableSetMap & TableFunctionMerge::getSourceDatabasesAndTables(ContextPtr context) const
|
||||
const TableFunctionMerge::DBToTableSetMap & TableFunctionMerge::getSourceDatabasesAndTables(ContextPtr context) const
|
||||
{
|
||||
if (source_databases_and_tables)
|
||||
return *source_databases_and_tables;
|
||||
@ -88,17 +88,10 @@ const TableFunctionMerge::DbToTableSetMap & TableFunctionMerge::getSourceDatabas
|
||||
auto databases = DatabaseCatalog::instance().getDatabases();
|
||||
|
||||
for (const auto & db : databases)
|
||||
{
|
||||
if (database_re.match(db.first))
|
||||
{
|
||||
auto source_tables = getMatchedTablesWithAccess(db.first, source_table_regexp, context);
|
||||
(*source_databases_and_tables)[db.first] = getMatchedTablesWithAccess(db.first, source_table_regexp, context);
|
||||
|
||||
if (!source_tables.empty())
|
||||
(*source_databases_and_tables)[db.first] = source_tables;
|
||||
}
|
||||
}
|
||||
|
||||
if ((*source_databases_and_tables).empty())
|
||||
if (source_databases_and_tables->empty())
|
||||
throwNoTablesMatchRegexp(source_database_name_or_regexp, source_table_regexp);
|
||||
}
|
||||
|
||||
|
@ -21,8 +21,8 @@ private:
|
||||
const char * getStorageTypeName() const override { return "Merge"; }
|
||||
|
||||
using TableSet = std::set<String>;
|
||||
using DbToTableSetMap = std::map<String, TableSet>;
|
||||
const DbToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const;
|
||||
using DBToTableSetMap = std::map<String, TableSet>;
|
||||
const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const;
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context);
|
||||
@ -30,7 +30,7 @@ private:
|
||||
String source_database_name_or_regexp;
|
||||
String source_table_regexp;
|
||||
bool database_is_regexp = false;
|
||||
mutable std::optional<DbToTableSetMap> source_databases_and_tables;
|
||||
mutable std::optional<DBToTableSetMap> source_databases_and_tables;
|
||||
};
|
||||
|
||||
|
||||
|
7
tests/queries/0_stateless/2024_merge_regexp_assert.sql
Normal file
7
tests/queries/0_stateless/2024_merge_regexp_assert.sql
Normal file
@ -0,0 +1,7 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
CREATE TABLE t (b UInt8) ENGINE = Memory;
|
||||
SELECT a FROM merge(REGEXP('.'), '^t$'); -- { serverError 47 }
|
||||
SELECT a FROM merge(REGEXP('\0'), '^t$'); -- { serverError 47 }
|
||||
SELECT a FROM merge(REGEXP('\0a'), '^t$'); -- { serverError 47 }
|
||||
SELECT a FROM merge(REGEXP('\0a'), '^$'); -- { serverError 36 }
|
||||
DROP TABLE t;
|
Loading…
Reference in New Issue
Block a user