use full match for storageMerge

This commit is contained in:
feng lv 2021-06-13 08:03:19 +00:00
parent 41b518012f
commit 1ffd380903
6 changed files with 207 additions and 198 deletions

View File

@ -2,6 +2,8 @@
#include <Common/PODArray.h>
#include <Common/OptimizedRegularExpression.h>
#include <Poco/String.h>
#define MIN_LENGTH_FOR_STRSTR 3
#define MAX_SUBPATTERNS 1024
@ -342,6 +344,22 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
}
}
template <bool thread_safe>
bool OptimizedRegularExpressionImpl<thread_safe>::fullMatch(const std::string & subject) const
{
if (is_trivial)
{
if (required_substring.empty())
return subject.empty();
if (is_case_insensitive)
return Poco::toLower(subject) == Poco::toLower(required_substring);
else
return subject == required_substring;
}
return RegexType::FullMatch(StringPieceType(subject.data(), subject.size()), *re2);
}
template <bool thread_safe>
bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, size_t subject_size) const

View File

@ -64,6 +64,8 @@ public:
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0);
bool fullMatch(const std::string & subject) const;
bool match(const std::string & subject) const
{
return match(subject.data(), subject.size());

View File

@ -144,8 +144,8 @@ StoragePtr StorageMerge::getFirstTable(F && predicate) const
{
while (iterator->isValid())
{
const auto & table = iterator->table();
if (table.get() != this && predicate(table))
const auto & table = iterator->table();
if (table.get() != this && predicate(table))
return table;
iterator->next();
@ -552,7 +552,7 @@ StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(Context
for (const auto & db : databases)
{
if (source_database_regexp->match(db.first))
if (source_database_regexp->fullMatch(db.first))
{
auto table_name_match = [this, &db](const String & table_name_) -> bool {
if (source_databases_and_tables)
@ -561,7 +561,7 @@ StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(Context
return source_tables.count(table_name_);
}
else
return source_table_regexp->match(table_name_);
return source_table_regexp->fullMatch(table_name_);
};
database_table_iterators.emplace_back(db.second->getTablesIterator(local_context, table_name_match));
}
@ -688,10 +688,6 @@ void registerStorageMerge(StorageFactory & factory)
String source_database_regexp = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
String table_name_regexp = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
/// If database argument is not String literal, we should not treat it as regexp
if (!engine_args[0]->as<ASTLiteral>())
source_database_regexp = "^" + source_database_regexp + "$";
return StorageMerge::create(
args.table_id, args.columns, args.comment, source_database_regexp, table_name_regexp, args.getContext());
});

View File

@ -49,15 +49,11 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr
" - name of source database and regexp for table names.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
auto db_arg = evaluateConstantExpressionForDatabaseName(args[0], context);
auto table_arg = evaluateConstantExpressionAsLiteral(args[1], context);
args[0] = evaluateConstantExpressionForDatabaseName(args[0], context);
args[1] = evaluateConstantExpressionAsLiteral(args[1], context);
source_database_regexp = db_arg->as<ASTLiteral &>().value.safeGet<String>();
source_table_regexp = table_arg->as<ASTLiteral &>().value.safeGet<String>();
/// If database argument is not String literal, we should not treat it as regexp
if (!args[0]->as<ASTLiteral>())
source_database_regexp = "^" + source_database_regexp + "$";
source_database_regexp = args[0]->as<ASTLiteral &>().value.safeGet<String>();
source_table_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>();
}
@ -66,11 +62,10 @@ const std::unordered_map<String, std::unordered_set<String>> & TableFunctionMerg
if (source_databases_and_tables)
return *source_databases_and_tables;
OptimizedRegularExpression database_re(source_database_regexp);
OptimizedRegularExpression table_re(source_table_regexp);
auto table_name_match = [&](const String & table_name_) { return table_re.match(table_name_); };
auto table_name_match = [&](const String & table_name_) { return table_re.fullMatch(table_name_); };
auto access = context->getAccess();
@ -78,7 +73,7 @@ const std::unordered_map<String, std::unordered_set<String>> & TableFunctionMerg
for (const auto & db : databases)
{
if (database_re.match(db.first))
if (database_re.fullMatch(db.first))
{
bool granted_show_on_all_tables = access->isGranted(AccessType::SHOW_TABLES, db.first);
bool granted_select_on_all_tables = access->isGranted(AccessType::SELECT, db.first);

View File

@ -1,146 +1,146 @@
CREATE TABLE t_merge as db.t ENGINE=Merge(^db, ^t)
SELECT _database, _table, n FROM db.t_merge ORDER BY _database, _table, n
db t 0
db t 1
db t 2
db t 3
db t 4
db t 5
db t 6
db t 7
db t 8
db t 9
db1 t1 0
db1 t1 1
db1 t1 2
db1 t1 3
db1 t1 4
db1 t1 5
db1 t1 6
db1 t1 7
db1 t1 8
db1 t1 9
db2 t2 0
db2 t2 1
db2 t2 2
db2 t2 3
db2 t2 4
db2 t2 5
db2 t2 6
db2 t2 7
db2 t2 8
db2 t2 9
db3 t3 0
db3 t3 1
db3 t3 2
db3 t3 3
db3 t3 4
db3 t3 5
db3 t3 6
db3 t3 7
db3 t3 8
db3 t3 9
CREATE TABLE t_merge as 01902_db.t ENGINE=Merge(^01902_db.*, ^t.*)
SELECT _database, _table, n FROM 01902_db.t_merge ORDER BY _database, _table, n
01902_db t 0
01902_db t 1
01902_db t 2
01902_db t 3
01902_db t 4
01902_db t 5
01902_db t 6
01902_db t 7
01902_db t 8
01902_db t 9
01902_db1 t1 0
01902_db1 t1 1
01902_db1 t1 2
01902_db1 t1 3
01902_db1 t1 4
01902_db1 t1 5
01902_db1 t1 6
01902_db1 t1 7
01902_db1 t1 8
01902_db1 t1 9
01902_db2 t2 0
01902_db2 t2 1
01902_db2 t2 2
01902_db2 t2 3
01902_db2 t2 4
01902_db2 t2 5
01902_db2 t2 6
01902_db2 t2 7
01902_db2 t2 8
01902_db2 t2 9
01902_db3 t3 0
01902_db3 t3 1
01902_db3 t3 2
01902_db3 t3 3
01902_db3 t3 4
01902_db3 t3 5
01902_db3 t3 6
01902_db3 t3 7
01902_db3 t3 8
01902_db3 t3 9
SELECT _database, _table, n FROM merge(^db, ^t) ORDER BY _database, _table, n
db t 0
db t 1
db t 2
db t 3
db t 4
db t 5
db t 6
db t 7
db t 8
db t 9
db t_merge 0
db t_merge 0
db t_merge 0
db t_merge 0
db t_merge 1
db t_merge 1
db t_merge 1
db t_merge 1
db t_merge 2
db t_merge 2
db t_merge 2
db t_merge 2
db t_merge 3
db t_merge 3
db t_merge 3
db t_merge 3
db t_merge 4
db t_merge 4
db t_merge 4
db t_merge 4
db t_merge 5
db t_merge 5
db t_merge 5
db t_merge 5
db t_merge 6
db t_merge 6
db t_merge 6
db t_merge 6
db t_merge 7
db t_merge 7
db t_merge 7
db t_merge 7
db t_merge 8
db t_merge 8
db t_merge 8
db t_merge 8
db t_merge 9
db t_merge 9
db t_merge 9
db t_merge 9
db1 t1 0
db1 t1 1
db1 t1 2
db1 t1 3
db1 t1 4
db1 t1 5
db1 t1 6
db1 t1 7
db1 t1 8
db1 t1 9
db2 t2 0
db2 t2 1
db2 t2 2
db2 t2 3
db2 t2 4
db2 t2 5
db2 t2 6
db2 t2 7
db2 t2 8
db2 t2 9
db3 t3 0
db3 t3 1
db3 t3 2
db3 t3 3
db3 t3 4
db3 t3 5
db3 t3 6
db3 t3 7
db3 t3 8
db3 t3 9
CREATE TABLE t_merge_1 as db.t ENGINE=Merge(currentDatabase(), ^t)
SELECT _database, _table, n FROM db.t_merge_1 ORDER BY _database, _table, n
db1 t1 0
db1 t1 1
db1 t1 2
db1 t1 3
db1 t1 4
db1 t1 5
db1 t1 6
db1 t1 7
db1 t1 8
db1 t1 9
SELECT _database, _table, n FROM merge(currentDatabase(), ^t) ORDER BY _database, _table, n
db1 t1 0
db1 t1 1
db1 t1 2
db1 t1 3
db1 t1 4
db1 t1 5
db1 t1 6
db1 t1 7
db1 t1 8
db1 t1 9
01902_db t 0
01902_db t 1
01902_db t 2
01902_db t 3
01902_db t 4
01902_db t 5
01902_db t 6
01902_db t 7
01902_db t 8
01902_db t 9
01902_db t_merge 0
01902_db t_merge 0
01902_db t_merge 0
01902_db t_merge 0
01902_db t_merge 1
01902_db t_merge 1
01902_db t_merge 1
01902_db t_merge 1
01902_db t_merge 2
01902_db t_merge 2
01902_db t_merge 2
01902_db t_merge 2
01902_db t_merge 3
01902_db t_merge 3
01902_db t_merge 3
01902_db t_merge 3
01902_db t_merge 4
01902_db t_merge 4
01902_db t_merge 4
01902_db t_merge 4
01902_db t_merge 5
01902_db t_merge 5
01902_db t_merge 5
01902_db t_merge 5
01902_db t_merge 6
01902_db t_merge 6
01902_db t_merge 6
01902_db t_merge 6
01902_db t_merge 7
01902_db t_merge 7
01902_db t_merge 7
01902_db t_merge 7
01902_db t_merge 8
01902_db t_merge 8
01902_db t_merge 8
01902_db t_merge 8
01902_db t_merge 9
01902_db t_merge 9
01902_db t_merge 9
01902_db t_merge 9
01902_db1 t1 0
01902_db1 t1 1
01902_db1 t1 2
01902_db1 t1 3
01902_db1 t1 4
01902_db1 t1 5
01902_db1 t1 6
01902_db1 t1 7
01902_db1 t1 8
01902_db1 t1 9
01902_db2 t2 0
01902_db2 t2 1
01902_db2 t2 2
01902_db2 t2 3
01902_db2 t2 4
01902_db2 t2 5
01902_db2 t2 6
01902_db2 t2 7
01902_db2 t2 8
01902_db2 t2 9
01902_db3 t3 0
01902_db3 t3 1
01902_db3 t3 2
01902_db3 t3 3
01902_db3 t3 4
01902_db3 t3 5
01902_db3 t3 6
01902_db3 t3 7
01902_db3 t3 8
01902_db3 t3 9
CREATE TABLE t_merge_1 as 01902_db.t ENGINE=Merge(currentDatabase(), ^t.*)
SELECT _database, _table, n FROM 01902_db.t_merge_1 ORDER BY _database, _table, n
01902_db1 t1 0
01902_db1 t1 1
01902_db1 t1 2
01902_db1 t1 3
01902_db1 t1 4
01902_db1 t1 5
01902_db1 t1 6
01902_db1 t1 7
01902_db1 t1 8
01902_db1 t1 9
SELECT _database, _table, n FROM merge(currentDatabase(), ^t.*) ORDER BY _database, _table, n
01902_db1 t1 0
01902_db1 t1 1
01902_db1 t1 2
01902_db1 t1 3
01902_db1 t1 4
01902_db1 t1 5
01902_db1 t1 6
01902_db1 t1 7
01902_db1 t1 8
01902_db1 t1 9

View File

@ -1,46 +1,44 @@
DROP DATABASE IF EXISTS db;
DROP DATABASE IF EXISTS db1;
DROP DATABASE IF EXISTS db2;
DROP DATABASE IF EXISTS db3;
DROP DATABASE IF EXISTS 01902_db;
DROP DATABASE IF EXISTS 01902_db1;
DROP DATABASE IF EXISTS 01902_db2;
DROP DATABASE IF EXISTS 01902_db3;
CREATE DATABASE db;
CREATE DATABASE db1;
CREATE DATABASE db2;
CREATE DATABASE db3;
CREATE DATABASE 01902_db;
CREATE DATABASE 01902_db1;
CREATE DATABASE 01902_db2;
CREATE DATABASE 01902_db3;
CREATE TABLE db.t (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE db1.t1 (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE db2.t2 (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE db3.t3 (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE 01902_db.t (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE 01902_db1.t1 (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE 01902_db2.t2 (n Int8) ENGINE=MergeTree ORDER BY n;
CREATE TABLE 01902_db3.t3 (n Int8) ENGINE=MergeTree ORDER BY n;
INSERT INTO db.t SELECT * FROM numbers(10);
INSERT INTO db1.t1 SELECT * FROM numbers(10);
INSERT INTO db2.t2 SELECT * FROM numbers(10);
INSERT INTO db3.t3 SELECT * FROM numbers(10);
INSERT INTO 01902_db.t SELECT * FROM numbers(10);
INSERT INTO 01902_db1.t1 SELECT * FROM numbers(10);
INSERT INTO 01902_db2.t2 SELECT * FROM numbers(10);
INSERT INTO 01902_db3.t3 SELECT * FROM numbers(10);
SELECT 'CREATE TABLE t_merge as db.t ENGINE=Merge(^db, ^t)';
CREATE TABLE db.t_merge as db.t ENGINE=Merge('^db', '^t');
SELECT 'CREATE TABLE t_merge as 01902_db.t ENGINE=Merge(^01902_db.*, ^t.*)';
CREATE TABLE 01902_db.t_merge as 01902_db.t ENGINE=Merge('^01902_db.*', '^t.*');
SELECT 'SELECT _database, _table, n FROM db.t_merge ORDER BY _database, _table, n';
SELECT _database, _table, n FROM db.t_merge ORDER BY _database, _table, n;
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge ORDER BY _database, _table, n';
SELECT _database, _table, n FROM 01902_db.t_merge ORDER BY _database, _table, n;
SELECT 'SELECT _database, _table, n FROM merge(^db, ^t) ORDER BY _database, _table, n';
SELECT _database, _table, n FROM merge('^db', '^t') ORDER BY _database, _table, n;
SELECT _database, _table, n FROM merge('^01902_db.*', '^t.*') ORDER BY _database, _table, n;
USE db1;
USE 01902_db1;
-- evaluated value of expression should not be treat as repr
SELECT 'CREATE TABLE t_merge_1 as db.t ENGINE=Merge(currentDatabase(), ^t)';
CREATE TABLE db.t_merge_1 as db.t ENGINE=Merge(currentDatabase(), '^t');
SELECT 'CREATE TABLE t_merge_1 as 01902_db.t ENGINE=Merge(currentDatabase(), ^t.*)';
CREATE TABLE 01902_db.t_merge_1 as 01902_db.t ENGINE=Merge(currentDatabase(), '^t.*');
SELECT 'SELECT _database, _table, n FROM db.t_merge_1 ORDER BY _database, _table, n';
SELECT _database, _table, n FROM db.t_merge_1 ORDER BY _database, _table, n;
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge_1 ORDER BY _database, _table, n';
SELECT _database, _table, n FROM 01902_db.t_merge_1 ORDER BY _database, _table, n;
-- evaluated value of expression should not be treat as repr
SELECT 'SELECT _database, _table, n FROM merge(currentDatabase(), ^t) ORDER BY _database, _table, n';
SELECT _database, _table, n FROM merge(currentDatabase(), '^t') ORDER BY _database, _table, n;
SELECT 'SELECT _database, _table, n FROM merge(currentDatabase(), ^t.*) ORDER BY _database, _table, n';
SELECT _database, _table, n FROM merge(currentDatabase(), '^t.*') ORDER BY _database, _table, n;
DROP DATABASE db;
DROP DATABASE db1;
DROP DATABASE db2;
DROP DATABASE db3;
DROP DATABASE 01902_db;
DROP DATABASE 01902_db1;
DROP DATABASE 01902_db2;
DROP DATABASE 01902_db3;