mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge pull request #18103 from ClickHouse/revert-17983-fix-access-rights-for-merge-table-function
Revert "Fix access rights required for the merge() table function."
This commit is contained in:
commit
063488d943
@ -84,22 +84,6 @@ StorageMerge::StorageMerge(
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
|
||||
StorageMerge::StorageMerge(
|
||||
const StorageID & table_id_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & source_database_,
|
||||
const Tables & tables_,
|
||||
const Context & context_)
|
||||
: IStorage(table_id_)
|
||||
, source_database(source_database_)
|
||||
, tables(tables_)
|
||||
, global_context(context_.getGlobalContext())
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
StoragePtr StorageMerge::getFirstTable(F && predicate) const
|
||||
{
|
||||
@ -455,12 +439,8 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & cont
|
||||
e.addMessage("while getting table iterator of Merge table. Maybe caused by two Merge tables that will endlessly try to read each other's data");
|
||||
throw;
|
||||
}
|
||||
|
||||
if (tables)
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(*tables, source_database);
|
||||
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp->match(table_name_); };
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); };
|
||||
return database->getTablesIterator(context, table_name_match);
|
||||
}
|
||||
|
||||
|
@ -48,8 +48,7 @@ public:
|
||||
|
||||
private:
|
||||
String source_database;
|
||||
std::optional<OptimizedRegularExpression> table_name_regexp;
|
||||
std::optional<Tables> tables;
|
||||
OptimizedRegularExpression table_name_regexp;
|
||||
const Context & global_context;
|
||||
|
||||
using StorageWithLockAndName = std::tuple<StoragePtr, TableLockHolder, String>;
|
||||
@ -76,13 +75,6 @@ protected:
|
||||
const String & table_name_regexp_,
|
||||
const Context & context_);
|
||||
|
||||
StorageMerge(
|
||||
const StorageID & table_id_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & source_database_,
|
||||
const Tables & source_tables_,
|
||||
const Context & context_);
|
||||
|
||||
Pipe createSources(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <TableFunctions/TableFunctionMerge.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
@ -23,6 +22,29 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_, const Context & context)
|
||||
{
|
||||
OptimizedRegularExpression table_name_regexp(table_name_regexp_);
|
||||
auto table_name_match = [&](const String & table_name) { return table_name_regexp.match(table_name); };
|
||||
|
||||
StoragePtr any_table;
|
||||
|
||||
{
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
auto iterator = database->getTablesIterator(context, table_name_match);
|
||||
|
||||
if (iterator->isValid())
|
||||
if (const auto & table = iterator->table())
|
||||
any_table = table;
|
||||
}
|
||||
|
||||
if (!any_table)
|
||||
throw Exception("Error while executing table function merge. In database " + source_database + " no one matches regular expression: "
|
||||
+ table_name_regexp_, ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical();
|
||||
}
|
||||
|
||||
void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context)
|
||||
{
|
||||
ASTs & args_func = ast_function->children;
|
||||
@ -46,46 +68,9 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Conte
|
||||
table_name_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
|
||||
|
||||
const Tables & TableFunctionMerge::getMatchingTables(const Context & context) const
|
||||
{
|
||||
if (tables)
|
||||
return *tables;
|
||||
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
|
||||
OptimizedRegularExpression re(table_name_regexp);
|
||||
auto table_name_match = [&](const String & table_name_) { return re.match(table_name_); };
|
||||
|
||||
auto access = context.getAccess();
|
||||
bool granted_show_on_all_tables = access->isGranted(AccessType::SHOW_TABLES, source_database);
|
||||
bool granted_select_on_all_tables = access->isGranted(AccessType::SELECT, source_database);
|
||||
|
||||
tables.emplace();
|
||||
for (auto it = database->getTablesIterator(context, table_name_match); it->isValid(); it->next())
|
||||
{
|
||||
if (!it->table())
|
||||
continue;
|
||||
bool granted_show = granted_show_on_all_tables || access->isGranted(AccessType::SHOW_TABLES, source_database, it->name());
|
||||
if (!granted_show)
|
||||
continue;
|
||||
if (!granted_select_on_all_tables)
|
||||
access->checkAccess(AccessType::SELECT, source_database, it->name());
|
||||
tables->emplace(it->name(), it->table());
|
||||
}
|
||||
|
||||
if (tables->empty())
|
||||
throw Exception("Error while executing table function merge. In database " + source_database + " no one matches regular expression: "
|
||||
+ table_name_regexp, ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return *tables;
|
||||
}
|
||||
|
||||
|
||||
ColumnsDescription TableFunctionMerge::getActualTableStructure(const Context & context) const
|
||||
{
|
||||
auto first_table = getMatchingTables(context).begin()->second;
|
||||
return ColumnsDescription{first_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical()};
|
||||
return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)};
|
||||
}
|
||||
|
||||
StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
||||
@ -94,7 +79,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, cons
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
getActualTableStructure(context),
|
||||
source_database,
|
||||
getMatchingTables(context),
|
||||
table_name_regexp,
|
||||
context);
|
||||
|
||||
res->startup();
|
||||
|
@ -19,13 +19,11 @@ private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const override;
|
||||
const char * getStorageTypeName() const override { return "Merge"; }
|
||||
|
||||
const Tables & getMatchingTables(const Context & context) const;
|
||||
ColumnsDescription getActualTableStructure(const Context & context) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, const Context & context) override;
|
||||
|
||||
String source_database;
|
||||
String table_name_regexp;
|
||||
mutable std::optional<Tables> tables;
|
||||
};
|
||||
|
||||
|
||||
|
@ -1,55 +0,0 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance')
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
instance.query("CREATE TABLE table1(x UInt32) ENGINE = MergeTree ORDER BY tuple()")
|
||||
instance.query("CREATE TABLE table2(x UInt32) ENGINE = MergeTree ORDER BY tuple()")
|
||||
instance.query("INSERT INTO table1 VALUES (1)")
|
||||
instance.query("INSERT INTO table2 VALUES (2)")
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def cleanup_after_test():
|
||||
try:
|
||||
yield
|
||||
finally:
|
||||
instance.query("DROP USER IF EXISTS A")
|
||||
|
||||
|
||||
def test_merge():
|
||||
select_query = "SELECT * FROM merge('default', 'table[0-9]+') ORDER BY x"
|
||||
assert instance.query(select_query) == "1\n2\n"
|
||||
|
||||
instance.query("CREATE USER A")
|
||||
assert "it's necessary to have the grant CREATE TEMPORARY TABLE ON *.*" in instance.query_and_get_error(select_query, user = 'A')
|
||||
|
||||
instance.query("GRANT CREATE TEMPORARY TABLE ON *.* TO A")
|
||||
assert "no one matches regular expression" in instance.query_and_get_error(select_query, user = 'A')
|
||||
|
||||
instance.query("GRANT SELECT ON default.table1 TO A")
|
||||
assert instance.query(select_query, user = 'A') == "1\n"
|
||||
|
||||
instance.query("GRANT SELECT ON default.* TO A")
|
||||
assert instance.query(select_query, user = 'A') == "1\n2\n"
|
||||
|
||||
instance.query("REVOKE SELECT ON default.table1 FROM A")
|
||||
assert instance.query(select_query, user = 'A') == "2\n"
|
||||
|
||||
instance.query("REVOKE ALL ON default.* FROM A")
|
||||
instance.query("GRANT SELECT ON default.table1 TO A")
|
||||
instance.query("GRANT INSERT ON default.table2 TO A")
|
||||
assert "it's necessary to have the grant SELECT ON default.table2" in instance.query_and_get_error(select_query, user = 'A')
|
Loading…
Reference in New Issue
Block a user