Merge pull request #29355 from ClickHouse/remove-trash-db-regexp

Fix assert in table function `merge` with database regexp
This commit is contained in:
alexey-milovidov 2021-09-27 05:50:13 +03:00 committed by GitHub
commit 5a7f30171a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 59 additions and 51 deletions

View File

@ -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:

View File

@ -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,
};

View File

@ -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() + "'",

View File

@ -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:

View File

@ -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
{

View File

@ -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);
}

View File

@ -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;

View File

@ -69,6 +69,6 @@ private:
mutable std::shared_mutex rocksdb_ptr_mx;
String rocksdb_dir;
void initDb();
void initDB();
};
}

View File

@ -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());

View File

@ -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(

View File

@ -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);
}

View File

@ -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;
};

View 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;