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:
Smita Kulkarni 2022-10-26 18:45:23 +02:00
parent efd47c9eff
commit 6a8fa2d4a5
8 changed files with 97 additions and 20 deletions

View File

@ -39,3 +39,7 @@ You cant 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
:::

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,3 @@
0
(123,'str')
0

View File

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