mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Add restrections for several sources of external dictionaries
This commit is contained in:
parent
4b75381852
commit
ea8994b9e4
@ -373,6 +373,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
Poco::File(user_files_path).createDirectories();
|
||||
}
|
||||
|
||||
{
|
||||
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/");
|
||||
global_context->setDictionariesLibPath(dictionaries_lib_path);
|
||||
Poco::File(dictionaries_lib_path).createDirectories();
|
||||
}
|
||||
|
||||
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
|
||||
throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
|
@ -210,7 +210,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool /* check_config */) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
|
||||
};
|
||||
|
@ -27,7 +27,8 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const
|
||||
const Context & context,
|
||||
bool check_source_config) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
const auto & layout_prefix = config_prefix + ".layout";
|
||||
@ -38,7 +39,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
|
||||
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
|
||||
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context, check_source_config);
|
||||
|
||||
const auto & layout_type = keys.front();
|
||||
|
||||
@ -57,7 +58,17 @@ DictionaryPtr DictionaryFactory::create(
|
||||
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
|
||||
{
|
||||
auto configurationFromAST = getDictionaryConfigurationFromAST(ast);
|
||||
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context);
|
||||
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context, true);
|
||||
}
|
||||
|
||||
bool DictionaryFactory::isComplex(const std::string & layout_type) const
|
||||
{
|
||||
auto found = layout_complexity.find(layout_type);
|
||||
|
||||
if (found != layout_complexity.end())
|
||||
return found->second;
|
||||
|
||||
throw Exception{"Unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
}
|
||||
|
||||
|
||||
|
@ -28,12 +28,16 @@ public:
|
||||
|
||||
static DictionaryFactory & instance();
|
||||
|
||||
/// Create dictionary from AbstractConfiguration parsed
|
||||
/// from xml-file on filesystem.
|
||||
DictionaryPtr create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const;
|
||||
const Context & context,
|
||||
bool check_source_config = false) const;
|
||||
|
||||
/// Create dictionary from DDL-query
|
||||
DictionaryPtr create(const std::string & name,
|
||||
const ASTCreateQuery & ast,
|
||||
const Context & context) const;
|
||||
@ -45,7 +49,7 @@ public:
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)>;
|
||||
|
||||
bool isComplex(const std::string & layout_type) const { return layout_complexity.at(layout_type); }
|
||||
bool isComplex(const std::string & layout_type) const;
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
|
||||
|
||||
|
@ -80,7 +80,8 @@ DictionarySourcePtr DictionarySourceFactory::create(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Context & context) const
|
||||
const Context & context,
|
||||
bool check_config) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
@ -95,7 +96,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
|
||||
{
|
||||
const auto & create_source = found->second;
|
||||
auto sample_block = createSampleBlock(dict_struct);
|
||||
return create_source(dict_struct, config, config_prefix, sample_block, context);
|
||||
return create_source(dict_struct, config, config_prefix, sample_block, context, check_config);
|
||||
}
|
||||
|
||||
throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
|
@ -31,7 +31,8 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context)>;
|
||||
const Context & context,
|
||||
bool check_config)>;
|
||||
|
||||
DictionarySourceFactory();
|
||||
|
||||
@ -42,7 +43,8 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Context & context) const;
|
||||
const Context & context,
|
||||
bool check_config) const;
|
||||
|
||||
private:
|
||||
using SourceRegistry = std::unordered_map<std::string, Creator>;
|
||||
|
@ -20,6 +20,10 @@ namespace DB
|
||||
{
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DICTIONARY_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
@ -217,12 +221,21 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
return std::make_unique<ExecutableDictionarySource>(dict_struct, config, config_prefix + ".executable", sample_block, context);
|
||||
/// Executable dictionaries may execute arbitrary commands.
|
||||
/// It's OK for dictionaries created by administrator from xml-file, but
|
||||
/// maybe dangerous for dictionaries created from DDL-queries.
|
||||
if (check_config)
|
||||
throw Exception("Dictionaries with Executable dictionary source is not allowed", ErrorCodes::DICTIONARY_ACCESS_DENIED);
|
||||
|
||||
return std::make_unique<ExecutableDictionarySource>(
|
||||
dict_struct, config, config_prefix + ".executable",
|
||||
sample_block, context);
|
||||
};
|
||||
factory.registerSource("executable", createTableSource);
|
||||
}
|
||||
|
@ -11,16 +11,31 @@ namespace DB
|
||||
{
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
|
||||
FileDictionarySource::FileDictionarySource(
|
||||
const std::string & filename_, const std::string & format_, Block & sample_block_, const Context & context_)
|
||||
: filename{filename_}, format{format_}, sample_block{sample_block_}, context(context_)
|
||||
const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, const Context & context_, bool check_config)
|
||||
: filepath{filepath_}
|
||||
, format{format_}
|
||||
, sample_block{sample_block_}
|
||||
, context(context_)
|
||||
{
|
||||
if (check_config)
|
||||
{
|
||||
const String user_files_path = context.getUserFilesPath();
|
||||
if (!startsWith(filepath, user_files_path))
|
||||
throw Exception("File path " + filepath + " is not inside " + user_files_path, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
|
||||
: filename{other.filename}
|
||||
: filepath{other.filepath}
|
||||
, format{other.format}
|
||||
, sample_block{other.sample_block}
|
||||
, context(other.context)
|
||||
@ -31,7 +46,7 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
|
||||
|
||||
BlockInputStreamPtr FileDictionarySource::loadAll()
|
||||
{
|
||||
auto in_ptr = std::make_unique<ReadBufferFromFile>(filename);
|
||||
auto in_ptr = std::make_unique<ReadBufferFromFile>(filepath);
|
||||
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
|
||||
last_modification = getLastModification();
|
||||
|
||||
@ -41,13 +56,13 @@ BlockInputStreamPtr FileDictionarySource::loadAll()
|
||||
|
||||
std::string FileDictionarySource::toString() const
|
||||
{
|
||||
return "File: " + filename + ' ' + format;
|
||||
return "File: " + filepath + ' ' + format;
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp FileDictionarySource::getLastModification() const
|
||||
{
|
||||
return Poco::File{filename}.getLastModified();
|
||||
return Poco::File{filepath}.getLastModified();
|
||||
}
|
||||
|
||||
void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
@ -56,15 +71,16 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
const auto filename = config.getString(config_prefix + ".file.path");
|
||||
const auto filepath = config.getString(config_prefix + ".file.path");
|
||||
const auto format = config.getString(config_prefix + ".file.format");
|
||||
|
||||
return std::make_unique<FileDictionarySource>(filename, format, sample_block, context);
|
||||
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context, check_config);
|
||||
};
|
||||
|
||||
factory.registerSource("file", createTableSource);
|
||||
|
@ -13,7 +13,8 @@ class Context;
|
||||
class FileDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
FileDictionarySource(const std::string & filename_, const std::string & format_, Block & sample_block_, const Context & context_);
|
||||
FileDictionarySource(const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, const Context & context_, bool check_config);
|
||||
|
||||
FileDictionarySource(const FileDictionarySource & other);
|
||||
|
||||
@ -47,7 +48,7 @@ public:
|
||||
private:
|
||||
Poco::Timestamp getLastModification() const;
|
||||
|
||||
const std::string filename;
|
||||
const std::string filepath;
|
||||
const std::string format;
|
||||
Block sample_block;
|
||||
const Context & context;
|
||||
|
@ -25,7 +25,8 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
const Context & context_)
|
||||
const Context & context_,
|
||||
bool check_config)
|
||||
: log(&Logger::get("HTTPDictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
@ -36,6 +37,10 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
, context(context_)
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context))
|
||||
{
|
||||
|
||||
if (check_config)
|
||||
context.getRemoteHostFilter().checkURL(Poco::URI(url));
|
||||
|
||||
const auto & credentials_prefix = config_prefix + ".credentials";
|
||||
|
||||
if (config.has(credentials_prefix))
|
||||
@ -187,12 +192,15 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
return std::make_unique<HTTPDictionarySource>(dict_struct, config, config_prefix + ".http", sample_block, context);
|
||||
return std::make_unique<HTTPDictionarySource>(
|
||||
dict_struct, config, config_prefix + ".http",
|
||||
sample_block, context, check_config);
|
||||
};
|
||||
factory.registerSource("http", createTableSource);
|
||||
}
|
||||
|
@ -25,7 +25,8 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
const Context & context_);
|
||||
const Context & context_,
|
||||
bool check_config);
|
||||
|
||||
HTTPDictionarySource(const HTTPDictionarySource & other);
|
||||
HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete;
|
||||
|
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int EXTERNAL_LIBRARY_ERROR;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
|
||||
@ -122,17 +123,28 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_)
|
||||
Block & sample_block_,
|
||||
const Context & context,
|
||||
bool check_config)
|
||||
: log(&Logger::get("LibraryDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, config_prefix{config_prefix_}
|
||||
, path{config.getString(config_prefix + ".path", "")}
|
||||
, sample_block{sample_block_}
|
||||
{
|
||||
|
||||
if (check_config)
|
||||
{
|
||||
const String dictionaries_lib_path = context.getDictionariesLibPath();
|
||||
if (!startsWith(path, dictionaries_lib_path))
|
||||
throw Exception("LibraryDictionarySource: Library path " + dictionaries_lib_path + " is not inside " + dictionaries_lib_path, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
if (!Poco::File(path).exists())
|
||||
throw Exception(
|
||||
"LibraryDictionarySource: Can't load lib " + toString() + ": " + Poco::File(path).path() + " - File doesn't exist",
|
||||
ErrorCodes::FILE_DOESNT_EXIST);
|
||||
|
||||
description.init(sample_block);
|
||||
library = std::make_shared<SharedLibrary>(path, RTLD_LAZY
|
||||
#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
|
||||
@ -286,9 +298,10 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context &) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block);
|
||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
|
||||
};
|
||||
factory.registerSource("library", createTableSource);
|
||||
}
|
||||
|
@ -32,7 +32,9 @@ public:
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_);
|
||||
Block & sample_block_,
|
||||
const Context & context,
|
||||
bool check_config);
|
||||
|
||||
LibraryDictionarySource(const LibraryDictionarySource & other);
|
||||
LibraryDictionarySource & operator=(const LibraryDictionarySource &) = delete;
|
||||
|
@ -15,7 +15,8 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_POCO_MONGODB
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb", sample_block);
|
||||
#else
|
||||
|
@ -20,7 +20,8 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_MYSQL
|
||||
return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block);
|
||||
#else
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_POCO_REDIS
|
||||
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
|
||||
#else
|
||||
|
@ -238,7 +238,8 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr {
|
||||
const Context & context,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
||||
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
|
||||
context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
|
||||
@ -262,7 +263,8 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & /* config */,
|
||||
const std::string & /* config_prefix */,
|
||||
Block & /* sample_block */,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
|
||||
|
@ -125,6 +125,7 @@ struct ContextShared
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
String flags_path; /// Path to the directory with some control flags for server maintenance.
|
||||
String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function.
|
||||
String dictionaries_lib_path; /// Path to the directory with user provided binaries and libraries for external dictionaries.
|
||||
ConfigurationPtr config; /// Global configuration settings.
|
||||
|
||||
Databases databases; /// List of databases and tables in them.
|
||||
@ -542,6 +543,12 @@ String Context::getUserFilesPath() const
|
||||
return shared->user_files_path;
|
||||
}
|
||||
|
||||
String Context::getDictionariesLibPath() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
return shared->dictionaries_lib_path;
|
||||
}
|
||||
|
||||
void Context::setPath(const String & path)
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -556,6 +563,9 @@ void Context::setPath(const String & path)
|
||||
|
||||
if (shared->user_files_path.empty())
|
||||
shared->user_files_path = shared->path + "user_files/";
|
||||
|
||||
if (shared->dictionaries_lib_path.empty())
|
||||
shared->dictionaries_lib_path = shared->path + "dictionaries_lib/";
|
||||
}
|
||||
|
||||
void Context::setTemporaryPath(const String & path)
|
||||
@ -576,6 +586,12 @@ void Context::setUserFilesPath(const String & path)
|
||||
shared->user_files_path = path;
|
||||
}
|
||||
|
||||
void Context::setDictionariesLibPath(const String & path)
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->dictionaries_lib_path = path;
|
||||
}
|
||||
|
||||
void Context::setConfig(const ConfigurationPtr & config)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -191,11 +191,13 @@ public:
|
||||
String getTemporaryPath() const;
|
||||
String getFlagsPath() const;
|
||||
String getUserFilesPath() const;
|
||||
String getDictionariesLibPath() const;
|
||||
|
||||
void setPath(const String & path);
|
||||
void setTemporaryPath(const String & path);
|
||||
void setFlagsPath(const String & path);
|
||||
void setUserFilesPath(const String & path);
|
||||
void setDictionariesLibPath(const String & path);
|
||||
|
||||
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
|
@ -19,7 +19,10 @@ ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
|
||||
ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config) const
|
||||
{
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, context);
|
||||
/// For dictionaries from databases (created with DDL qureies) we have to perform
|
||||
/// additional checks, so we identify them here.
|
||||
bool dictionary_from_database = !key_in_config.empty();
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, context, dictionary_from_database);
|
||||
}
|
||||
|
||||
void ExternalDictionariesLoader::addConfigRepository(
|
||||
|
@ -16,4 +16,8 @@
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
|
||||
<remote_url_allow_hosts>
|
||||
<host>node1</host>
|
||||
</remote_url_allow_hosts>
|
||||
</yandex>
|
||||
|
@ -182,3 +182,35 @@ def test_conflicting_name(started_cluster):
|
||||
|
||||
# old version still works
|
||||
node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
|
||||
|
||||
def test_http_dictionary_restrictions(started_cluster):
|
||||
try:
|
||||
node3.query("""
|
||||
CREATE DICTIONARY test.restricted_http_dictionary (
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(HTTP(URL 'http://somehost.net' FORMAT TabSeparated))
|
||||
LIFETIME(1)
|
||||
""")
|
||||
node3.query("SELECT dictGetString('test.restricted_http_dictionary', 'value', toUInt64(1))")
|
||||
except QueryRuntimeException as ex:
|
||||
assert 'is not allowed in config.xml' in str(ex)
|
||||
|
||||
def test_file_dictionary_restrictions(started_cluster):
|
||||
try:
|
||||
node3.query("""
|
||||
CREATE DICTIONARY test.restricted_file_dictionary (
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(FILE(PATH '/usr/bin/cat' FORMAT TabSeparated))
|
||||
LIFETIME(1)
|
||||
""")
|
||||
node3.query("SELECT dictGetString('test.restricted_file_dictionary', 'value', toUInt64(1))")
|
||||
except QueryRuntimeException as ex:
|
||||
assert 'is not inside' in str(ex)
|
||||
|
@ -0,0 +1 @@
|
||||
Ok.
|
@ -0,0 +1,21 @@
|
||||
DROP DATABASE IF EXISTS dictdb;
|
||||
|
||||
CREATE DATABASE dictdb ENGINE=Ordinary;
|
||||
|
||||
CREATE DICTIONARY dictdb.restricted_dict (
|
||||
key UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(EXECUTABLE(COMMAND 'echo -E "1\thello"' FORMAT TabSeparated))
|
||||
LIFETIME(MIN 0 MAX 1)
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
-- because of lazy load we can check only in dictGet query
|
||||
select dictGetString('dictdb.restricted_dict', 'value', toUInt64(1)); -- {serverError 482}
|
||||
|
||||
select 'Ok.';
|
||||
|
||||
DROP DICTIONARY IF EXISTS dictdb.restricted_dict;
|
||||
|
||||
DROP DATABASE IF EXISTS dictdb;
|
Loading…
Reference in New Issue
Block a user