mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Added new field allow_readonly in system.table_functions to allow using table functions in readonly mode.
Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. * Added TableFunctionProperties similar to AggregateFunctionProperties. * The functions allowed in readonly mode are in this set table_functions_allowed_in_readonly_mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions.
This commit is contained in:
parent
efd47c9eff
commit
6a8fa2d4a5
@ -39,3 +39,7 @@ You can’t use table functions if the [allow_ddl](../../operations/settings/per
|
||||
| [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. |
|
||||
| [sqlite](../../sql-reference/table-functions/sqlite.md) | Creates a [sqlite](../../engines/table-engines/integrations/sqlite.md)-engine table. |
|
||||
|
||||
:::note
|
||||
Only these table functions are enabled in readonly mode :
|
||||
null, view, viewIfPermitted, numbers, numbers_mt, generateRandom, values, cluster, clusterAllReplicas
|
||||
:::
|
@ -1,16 +1,23 @@
|
||||
#include <Storages/System/StorageSystemTableFunctions.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FUNCTION;
|
||||
}
|
||||
|
||||
NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes()
|
||||
{
|
||||
return
|
||||
{
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"description", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
{
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"description", std::make_shared<DataTypeString>()},
|
||||
{"allow_readonly", std::make_shared<DataTypeUInt8>()}
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const
|
||||
@ -20,7 +27,15 @@ void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, Context
|
||||
for (const auto & function_name : functions_names)
|
||||
{
|
||||
res_columns[0]->insert(function_name);
|
||||
res_columns[1]->insert(factory.getDocumentation(function_name).description);
|
||||
|
||||
auto properties = factory.tryGetProperties(function_name);
|
||||
if (properties)
|
||||
{
|
||||
res_columns[1]->insert(properties->documentation.description);
|
||||
res_columns[2]->insert(properties->allow_readonly);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", function_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Storages/StorageTableFunction.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -25,8 +26,8 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte
|
||||
ProfileEvents::increment(ProfileEvents::TableFunctionExecute);
|
||||
|
||||
AccessFlags required_access = getSourceAccessType();
|
||||
String function_name = getName();
|
||||
if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted"))
|
||||
auto table_function_properties = TableFunctionFactory::instance().tryGetProperties(getName());
|
||||
if (!(table_function_properties && table_function_properties->allow_readonly))
|
||||
required_access |= AccessType::CREATE_TEMPORARY_TABLE;
|
||||
context->checkAccess(required_access);
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Access/Common/AccessType.h>
|
||||
#include <Common/Documentation.h>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
@ -79,6 +80,14 @@ private:
|
||||
virtual const char * getStorageTypeName() const = 0;
|
||||
};
|
||||
|
||||
/// Properties of table function that are independent of argument types and parameters.
|
||||
struct TableFunctionProperties
|
||||
{
|
||||
Documentation documentation;
|
||||
bool allow_readonly = false;
|
||||
};
|
||||
|
||||
|
||||
using TableFunctionPtr = std::shared_ptr<ITableFunction>;
|
||||
|
||||
|
||||
|
@ -16,16 +16,22 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static const NameSet table_functions_allowed_in_readonly_mode
|
||||
{
|
||||
"null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas"
|
||||
};
|
||||
|
||||
void TableFunctionFactory::registerFunction(
|
||||
const std::string & name, TableFunctionCreator creator, Documentation doc, CaseSensitiveness case_sensitiveness)
|
||||
{
|
||||
if (!table_functions.emplace(name, TableFunctionFactoryData{creator, doc}).second)
|
||||
bool allowed_in_readonly_mode = table_functions_allowed_in_readonly_mode.contains(name);
|
||||
|
||||
if (!table_functions.emplace(name, TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second)
|
||||
throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive
|
||||
&& !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, doc}).second)
|
||||
&& !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second)
|
||||
throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
@ -59,13 +65,13 @@ TableFunctionPtr TableFunctionFactory::tryGet(
|
||||
auto it = table_functions.find(name);
|
||||
if (table_functions.end() != it)
|
||||
{
|
||||
res = it->second.first();
|
||||
res = it->second.creator();
|
||||
}
|
||||
else
|
||||
{
|
||||
it = case_insensitive_table_functions.find(Poco::toLower(name));
|
||||
if (case_insensitive_table_functions.end() != it)
|
||||
res = it->second.first();
|
||||
res = it->second.creator();
|
||||
}
|
||||
|
||||
if (!res)
|
||||
@ -86,13 +92,29 @@ bool TableFunctionFactory::isTableFunctionName(const std::string & name) const
|
||||
return table_functions.contains(name);
|
||||
}
|
||||
|
||||
Documentation TableFunctionFactory::getDocumentation(const std::string & name) const
|
||||
std::optional<TableFunctionProperties> TableFunctionFactory::tryGetProperties(const String & name) const
|
||||
{
|
||||
auto it = table_functions.find(name);
|
||||
if (it == table_functions.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", name);
|
||||
return tryGetPropertiesImpl(name);
|
||||
}
|
||||
|
||||
return it->second.second;
|
||||
std::optional<TableFunctionProperties> TableFunctionFactory::tryGetPropertiesImpl(const String & name_param) const
|
||||
{
|
||||
String name = getAliasToOrName(name_param);
|
||||
Value found;
|
||||
|
||||
/// Find by exact match.
|
||||
if (auto it = table_functions.find(name); it != table_functions.end())
|
||||
{
|
||||
found = it->second;
|
||||
}
|
||||
|
||||
if (auto jt = case_insensitive_table_functions.find(Poco::toLower(name)); jt != case_insensitive_table_functions.end())
|
||||
found = jt->second;
|
||||
|
||||
if (found.creator)
|
||||
return found.properties;
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
TableFunctionFactory & TableFunctionFactory::instance()
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
#include <Common/Documentation.h>
|
||||
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
@ -18,7 +17,24 @@ namespace DB
|
||||
class Context;
|
||||
|
||||
using TableFunctionCreator = std::function<TableFunctionPtr()>;
|
||||
using TableFunctionFactoryData = std::pair<TableFunctionCreator, Documentation>;
|
||||
|
||||
struct TableFunctionFactoryData
|
||||
{
|
||||
TableFunctionCreator creator;
|
||||
TableFunctionProperties properties;
|
||||
|
||||
TableFunctionFactoryData() = default;
|
||||
TableFunctionFactoryData(const TableFunctionFactoryData &) = default;
|
||||
TableFunctionFactoryData & operator = (const TableFunctionFactoryData &) = default;
|
||||
|
||||
template <typename Creator>
|
||||
requires (!std::is_same_v<Creator, TableFunctionFactoryData>)
|
||||
TableFunctionFactoryData(Creator creator_, TableFunctionProperties properties_ = {}) /// NOLINT
|
||||
: creator(std::forward<Creator>(creator_)), properties(std::move(properties_))
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Lets you get a table function by its name.
|
||||
*/
|
||||
@ -48,7 +64,7 @@ public:
|
||||
/// Returns nullptr if not found.
|
||||
TableFunctionPtr tryGet(const std::string & name, ContextPtr context) const;
|
||||
|
||||
Documentation getDocumentation(const std::string & name) const;
|
||||
std::optional<TableFunctionProperties> tryGetProperties(const String & name) const;
|
||||
|
||||
bool isTableFunctionName(const std::string & name) const;
|
||||
|
||||
@ -61,6 +77,8 @@ private:
|
||||
|
||||
String getFactoryName() const override { return "TableFunctionFactory"; }
|
||||
|
||||
std::optional<TableFunctionProperties> tryGetPropertiesImpl(const String & name) const;
|
||||
|
||||
TableFunctions table_functions;
|
||||
TableFunctions case_insensitive_table_functions;
|
||||
};
|
||||
|
@ -0,0 +1,3 @@
|
||||
0
|
||||
(123,'str')
|
||||
0
|
@ -0,0 +1,5 @@
|
||||
SELECT * from numbers(1);
|
||||
select * from format(JSONEachRow, '{"x" : [123, "str"]}');
|
||||
SELECT * from numbers(1) SETTINGS readonly=1;
|
||||
select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY }
|
||||
set readonly=0;
|
Loading…
Reference in New Issue
Block a user