Merge pull request #13743 from ClickHouse/fix-stress-test

Fix server restart after stress test
This commit is contained in:
alexey-milovidov 2020-08-15 12:16:01 +03:00 committed by GitHub
commit b0ffde5825
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 46 additions and 26 deletions

View File

@ -153,7 +153,6 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
if (isTableExist(dictionary_name, global_context))
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", dict_id.getFullTableName());
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp";
String statement = getObjectDefinitionFromCreateQuery(query);

View File

@ -18,6 +18,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
[[maybe_unused]] const std::string & config_prefix,
[[maybe_unused]] Block & sample_block,
const Context & /* context */,
const std::string & /* default_database */,
bool /*check_config*/) -> DictionarySourcePtr
{
#if USE_CASSANDRA

View File

@ -53,7 +53,8 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
const std::string & path_to_settings,
const std::string & config_prefix,
const Block & sample_block_,
const Context & context_)
const Context & context_,
const std::string & default_database)
: update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, host{config.getString(config_prefix + ".host")}
@ -61,7 +62,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
, secure(config.getBool(config_prefix + ".secure", false))
, user{config.getString(config_prefix + ".user", "")}
, password{config.getString(config_prefix + ".password", "")}
, db{config.getString(config_prefix + ".db", "")}
, db{config.getString(config_prefix + ".db", default_database)}
, table{config.getString(config_prefix + ".table")}
, where{config.getString(config_prefix + ".where", "")}
, update_field{config.getString(config_prefix + ".update_field", "")}
@ -226,9 +227,11 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & default_database,
bool /* check_config */) -> DictionarySourcePtr
{
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context);
return std::make_unique<ClickHouseDictionarySource>(
dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context, default_database);
};
factory.registerSource("clickhouse", create_table_source);
}

View File

@ -24,7 +24,8 @@ public:
const std::string & path_to_settings,
const std::string & config_prefix,
const Block & sample_block_,
const Context & context);
const Context & context,
const std::string & default_database);
/// copy-constructor is provided in order to support cloneability
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);

View File

@ -42,7 +42,8 @@ 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, check_source_config);
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(
name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), check_source_config);
LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name);
const auto & layout_type = keys.front();

View File

@ -80,6 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
const std::string & config_prefix,
const DictionaryStructure & dict_struct,
const Context & context,
const std::string & default_database,
bool check_config) const
{
Poco::Util::AbstractConfiguration::Keys keys;
@ -96,7 +97,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, check_config);
return create_source(dict_struct, config, config_prefix, sample_block, context, default_database, check_config);
}
throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};

View File

@ -26,12 +26,16 @@ class DictionarySourceFactory : private boost::noncopyable
public:
static DictionarySourceFactory & instance();
/// 'default_database' - the database where dictionary itself was created.
/// It is used as default_database for ClickHouse dictionary source when no explicit database was specified.
/// Does not make sense for other sources.
using Creator = std::function<DictionarySourcePtr(
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & default_database,
bool check_config)>;
DictionarySourceFactory();
@ -44,6 +48,7 @@ public:
const std::string & config_prefix,
const DictionaryStructure & dict_struct,
const Context & context,
const std::string & default_database,
bool check_config) const;
private:

View File

@ -220,6 +220,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)

View File

@ -76,6 +76,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)

View File

@ -197,6 +197,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)

View File

@ -298,6 +298,7 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
{
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);

View File

@ -14,6 +14,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
const std::string & root_config_prefix,
Block & sample_block,
const Context &,
const std::string & /* default_database */,
bool /* check_config */)
{
const auto config_prefix = root_config_prefix + ".mongodb";

View File

@ -19,6 +19,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & /* context */,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
#if USE_MYSQL
return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block);

View File

@ -13,6 +13,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory)
const String & config_prefix,
Block & sample_block,
const Context & /* context */,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
};

View File

@ -275,6 +275,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
const std::string & config_prefix,
Block & sample_block,
const Context & context,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
#if USE_ODBC
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
@ -300,6 +301,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
const std::string & /* config_prefix */,
Block & /* sample_block */,
const Context & /* context */,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.",
ErrorCodes::SUPPORT_IS_DISABLED};

View File

@ -402,7 +402,11 @@ void buildConfigurationFromFunctionWithKeyValueArguments(
* </mysql>
* </source>
*/
void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source, const ASTDictionarySettings * settings)
void buildSourceConfiguration(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTFunctionWithKeyValueArguments * source,
const ASTDictionarySettings * settings)
{
AutoPtr<Element> outer_element(doc->createElement("source"));
root->appendChild(outer_element);
@ -498,7 +502,9 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type);
auto all_attr_names_and_types = buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_attrs);
auto all_attr_names_and_types = buildDictionaryAttributesConfiguration(
xml_document, structure_element, query.dictionary_attributes_list, pk_attrs);
checkPrimaryKey(all_attr_names_and_types, pk_attrs);
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list);

View File

@ -1,11 +1,11 @@
CREATE TABLE test_01083.file\n(\n `n` Int8\n)\nENGINE = File(\'TSVWithNamesAndTypes\')
CREATE TABLE test_01083.buffer\n(\n `n` Int8\n)\nENGINE = Buffer(\'test_01083\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000)
CREATE TABLE test_01083.merge\n(\n `n` Int8\n)\nENGINE = Merge(\'test_01083\', \'distributed\')
CREATE TABLE test_01083.merge_tf AS merge(\'test_01083\', \'.*\')
CREATE TABLE test_01083.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'test_01083\', \'file\')
CREATE TABLE test_01083.distributed_tf AS cluster(\'test_shard_localhost\', \'test_01083\', \'buffer\')
CREATE TABLE test_01083.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+test_01083.merge+format+CSV\', \'CSV\')
CREATE TABLE test_01083.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'test_01083\', \'view\')))
CREATE VIEW test_01083.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM test_01083.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM test_01083.file
CREATE DICTIONARY test_01083.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\' DB \'test_01083\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1))
CREATE TABLE default.file\n(\n `n` Int8\n)\nENGINE = File(\'TSVWithNamesAndTypes\')
CREATE TABLE default.buffer\n(\n `n` Int8\n)\nENGINE = Buffer(\'default\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000)
CREATE TABLE default.merge\n(\n `n` Int8\n)\nENGINE = Merge(\'default\', \'distributed\')
CREATE TABLE default.merge_tf AS merge(\'default\', \'.*\')
CREATE TABLE default.distributed\n(\n `n` Int8\n)\nENGINE = Distributed(\'test_shard_localhost\', \'default\', \'file\')
CREATE TABLE default.distributed_tf AS cluster(\'test_shard_localhost\', \'default\', \'buffer\')
CREATE TABLE default.url\n(\n `n` UInt64,\n `col` String\n)\nENGINE = URL(\'https://localhost:8443/?query=select+n,+_table+from+default.merge+format+CSV\', \'CSV\')
CREATE TABLE default.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_shard_localhost\', remote(\'127.0.0.{1..4}\', \'default\', \'view\')))
CREATE VIEW default.view\n(\n `n` Int64\n) AS\nSELECT toInt64(n) AS n\nFROM \n(\n SELECT toString(n) AS n\n FROM default.merge\n WHERE _table != \'qwerty\'\n ORDER BY _table ASC\n)\nUNION ALL\nSELECT *\nFROM default.file
CREATE DICTIONARY default.dict\n(\n `n` UInt64,\n `col` String DEFAULT \'42\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9440 SECURE 1 USER \'default\' TABLE \'url\'))\nLIFETIME(MIN 0 MAX 1)\nLAYOUT(CACHE(SIZE_IN_CELLS 1))
16

View File

@ -1,7 +1,3 @@
DROP DATABASE IF EXISTS test_01083;
CREATE DATABASE test_01083;
USE test_01083;
CREATE TABLE file (n Int8) ENGINE = File(upper('tsv') || 'WithNames' || 'AndTypes');
CREATE TABLE buffer (n Int8) ENGINE = Buffer(currentDatabase(), file, 16, 10, 200, 10000, 1000000, 10000000, 1000000000);
CREATE TABLE merge (n Int8) ENGINE = Merge('', lower('DISTRIBUTED'));
@ -28,7 +24,7 @@ CREATE VIEW view AS SELECT toInt64(n) as n FROM (SELECT toString(n) as n from me
SELECT nonexistentsomething; -- { serverError 47 }
CREATE DICTIONARY dict (n UInt64, col String DEFAULT '42') PRIMARY KEY n
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9440 SECURE 1 USER 'default' TABLE 'url' DB 'test_01083')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1));
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9440 SECURE 1 USER 'default' TABLE 'url')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1));
-- dict --> url --> merge |-> distributed -> file (1)
-- |-> distributed_tf -> buffer -> file (1)
@ -72,5 +68,3 @@ INSERT INTO buffer VALUES (1);
-- | |-> file (1)
-- |-> remote(127.0.0.2) --> ...
SELECT sum(n) from rich_syntax;
DROP DATABASE test_01083;