Add restrections for several sources of external dictionaries

This commit is contained in:
alesapin 2019-12-10 20:27:29 +03:00
parent 4b75381852
commit ea8994b9e4
24 changed files with 200 additions and 37 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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