From 18dd1e1f142ad5b2161674c96e6ee6fd159f6b16 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Jul 2018 19:10:57 +0300 Subject: [PATCH 01/17] CLICKHOUSE-3772: Fixes in system tables formats, data_type_families, aggregate_function_combinators --- dbms/src/DataTypes/DataTypeFactory.cpp | 65 ++++++++++- dbms/src/DataTypes/DataTypeFactory.h | 23 +++- dbms/src/DataTypes/DataTypeFixedString.cpp | 2 +- dbms/src/DataTypes/DataTypeString.cpp | 20 ++-- dbms/src/DataTypes/DataTypesNumber.cpp | 14 +-- ...ringColumns.h => IStorageSystemOneBlock.h} | 12 +-- ...rageSystemAggregateFunctionCombinators.cpp | 4 + ...torageSystemAggregateFunctionCombinators.h | 15 ++- .../System/StorageSystemCollations.cpp | 4 +- .../Storages/System/StorageSystemCollations.h | 12 ++- .../System/StorageSystemDataTypeFamilies.cpp | 101 ++++++++++-------- .../System/StorageSystemDataTypeFamilies.h | 31 +++++- .../Storages/System/StorageSystemFormats.cpp | 19 +--- .../Storages/System/StorageSystemFormats.h | 18 ++-- .../System/StorageSystemTableFunctions.h | 11 +- 15 files changed, 229 insertions(+), 122 deletions(-) rename dbms/src/Storages/System/{IStorageSystemWithStringColumns.h => IStorageSystemOneBlock.h} (74%) diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index f1a12d75868..72f34b23836 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -59,13 +59,19 @@ DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & para return it->second(parameters); } + String family_name_lowercase = Poco::toLower(family_name); + { - String family_name_lowercase = Poco::toLower(family_name); DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase); if (case_insensitive_data_types.end() != it) return it->second(parameters); } + if (auto it = aliases.find(family_name); it != aliases.end()) + return get(it->second, parameters); + else if (auto it = case_insensitive_aliases.find(family_name_lowercase); it != case_insensitive_aliases.end()) + return get(it->second, parameters); + throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE); } @@ -76,11 +82,16 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat throw Exception("DataTypeFactory: the data type family " + family_name + " has been provided " " a null constructor", ErrorCodes::LOGICAL_ERROR); + String family_name_lowercase = Poco::toLower(family_name); + + if (aliases.count(family_name) || case_insensitive_aliases.count(family_name_lowercase)) + throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is already registered as alias", + ErrorCodes::LOGICAL_ERROR); + if (!data_types.emplace(family_name, creator).second) throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - String family_name_lowercase = Poco::toLower(family_name); if (case_sensitiveness == CaseInsensitive && !case_insensitive_data_types.emplace(family_name_lowercase, creator).second) @@ -88,6 +99,27 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat ErrorCodes::LOGICAL_ERROR); } +void DataTypeFactory::registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) { + String real_type_dict_name; + if (data_types.count(real_name)) + real_type_dict_name = real_name; + else if (auto type_name_lowercase = Poco::toLower(real_name); case_insensitive_data_types.count(type_name_lowercase)) + real_type_dict_name = type_name_lowercase; + else + throw Exception("DataTypeFactory: can't create alias '" + alias_name + "' the data type family '" + real_name + "' is not registered", ErrorCodes::LOGICAL_ERROR); + + String alias_name_lowercase = Poco::toLower(alias_name); + + if (data_types.count(alias_name) || case_insensitive_data_types.count(alias_name_lowercase)) + throw Exception("DataTypeFactory: the alias name " + alias_name + " is already registered as datatype", ErrorCodes::LOGICAL_ERROR); + + if (case_sensitiveness == CaseInsensitive) + if(!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second) + throw Exception("DataTypeFactory: case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + + if(!aliases.emplace(alias_name, real_type_dict_name).second) + throw Exception("DataTypeFactory: alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); +} void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness) { @@ -103,6 +135,35 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator }, case_sensitiveness); } +std::vector DataTypeFactory::getAllDataTypeNames() const +{ + std::vector result; + auto getter = [] (const auto& pair) { return pair.first; }; + std::transform(data_types.begin(), data_types.end(), std::back_inserter(result), getter); + std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter); + return result; +} + +bool DataTypeFactory::isCaseInsensitive(const String & name) const +{ + String name_lowercase = Poco::toLower(name); + return case_insensitive_data_types.count(name_lowercase) || case_insensitive_aliases.count(name_lowercase); +} + +const String & DataTypeFactory::aliasTo(const String & name) const +{ + if (auto it = aliases.find(name); it != aliases.end()) + return it->second; + else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end()) + return it->second; + + throw Exception("DataTypeFactory: the data type '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR); +} + + +bool DataTypeFactory::isAlias(const String & name) const { + return aliases.count(name) || case_insensitive_aliases.count(name); +} void registerDataTypeNumbers(DataTypeFactory & factory); void registerDataTypeDate(DataTypeFactory & factory); diff --git a/dbms/src/DataTypes/DataTypeFactory.h b/dbms/src/DataTypes/DataTypeFactory.h index e6c873ba724..423ff6acd1e 100644 --- a/dbms/src/DataTypes/DataTypeFactory.h +++ b/dbms/src/DataTypes/DataTypeFactory.h @@ -25,6 +25,7 @@ private: using Creator = std::function; using SimpleCreator = std::function; using DataTypesDictionary = std::unordered_map; + using AliasMap = std::unordered_map; // alias -> original type public: DataTypePtr get(const String & full_name) const; @@ -44,10 +45,18 @@ public: /// Register a simple data type, that have no parameters. void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - const DataTypesDictionary & getAllDataTypes() const - { - return data_types; - } + /** Register additional name for datatype. + * real_name datatype have to be already registered. + */ + void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); + + std::vector getAllDataTypeNames() const; + + bool isCaseInsensitive(const String & name) const; + + const String & aliasTo(const String & name) const; + + bool isAlias(const String & name) const; private: DataTypesDictionary data_types; @@ -55,6 +64,12 @@ private: /// Case insensitive data types will be additionally added here with lowercased name. DataTypesDictionary case_insensitive_data_types; + /// Alias map to data_types from previous two maps + AliasMap aliases; + + /// Case insensitive aliases + AliasMap case_insensitive_aliases; + DataTypeFactory(); friend class ext::singleton; }; diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index 05fdd34c464..ad875c4f85e 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -231,7 +231,7 @@ void registerDataTypeFixedString(DataTypeFactory & factory) factory.registerDataType("FixedString", create); /// Compatibility alias. - factory.registerDataType("BINARY", create, DataTypeFactory::CaseInsensitive); + factory.registerAlias("BINARY", "FixedString", DataTypeFactory::CaseInsensitive); } } diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 4ffda6f2099..671d1b2d3a5 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -312,16 +312,16 @@ void registerDataTypeString(DataTypeFactory & factory) /// These synonims are added for compatibility. - factory.registerSimpleDataType("CHAR", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("VARCHAR", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("TEXT", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("TINYTEXT", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("MEDIUMTEXT", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("LONGTEXT", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("BLOB", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("TINYBLOB", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("MEDIUMBLOB", creator, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("LONGBLOB", creator, DataTypeFactory::CaseInsensitive); + factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive); } } diff --git a/dbms/src/DataTypes/DataTypesNumber.cpp b/dbms/src/DataTypes/DataTypesNumber.cpp index 72861eff3ac..254d6ba6852 100644 --- a/dbms/src/DataTypes/DataTypesNumber.cpp +++ b/dbms/src/DataTypes/DataTypesNumber.cpp @@ -22,13 +22,13 @@ void registerDataTypeNumbers(DataTypeFactory & factory) /// These synonims are added for compatibility. - factory.registerSimpleDataType("TINYINT", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("SMALLINT", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("INT", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("INTEGER", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("BIGINT", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("FLOAT", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); - factory.registerSimpleDataType("DOUBLE", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); + factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive); + factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); } } diff --git a/dbms/src/Storages/System/IStorageSystemWithStringColumns.h b/dbms/src/Storages/System/IStorageSystemOneBlock.h similarity index 74% rename from dbms/src/Storages/System/IStorageSystemWithStringColumns.h rename to dbms/src/Storages/System/IStorageSystemOneBlock.h index 08e2f0a7bf5..ad50b43b975 100644 --- a/dbms/src/Storages/System/IStorageSystemWithStringColumns.h +++ b/dbms/src/Storages/System/IStorageSystemOneBlock.h @@ -14,21 +14,15 @@ class Context; /** Base class for system tables whose all columns have String type. */ template -class IStorageSystemWithStringColumns : public IStorage +class IStorageSystemOneBlock : public IStorage { protected: virtual void fillData(MutableColumns & res_columns) const = 0; public: - IStorageSystemWithStringColumns(const String & name_) : name(name_) + IStorageSystemOneBlock(const String & name_) : name(name_) { - auto names = Self::getColumnNames(); - NamesAndTypesList name_list; - for (const auto & name : names) - { - name_list.push_back(NameAndTypePair{name, std::make_shared()}); - } - setColumns(ColumnsDescription(name_list)); + setColumns(ColumnsDescription(Self::getNamesAndTypes())); } std::string getTableName() const override diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp index 9dd106ce2d7..2f23d66aac2 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp @@ -9,6 +9,10 @@ void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_co for (const auto & pair : combinators) { res_columns[0]->insert(pair.first); + if (pair.first != "Null") + res_columns[1]->insert(UInt64(0)); + else + res_columns[1]->insert(UInt64(1)); } } } diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h index 097fe93666e..56aaabad463 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h @@ -1,26 +1,31 @@ #pragma once -#include +#include +#include +#include #include namespace DB { class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper, - public IStorageSystemWithStringColumns + public IStorageSystemOneBlock { protected: void fillData(MutableColumns & res_columns) const override; public: - using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns; + using IStorageSystemOneBlock::IStorageSystemOneBlock; std::string getName() const override { return "SystemAggregateFunctionCombinators"; } - static std::vector getColumnNames() + static NamesAndTypesList getNamesAndTypes() { - return {"name"}; + return { + {"name", std::make_shared()}, + {"is_internal", std::make_shared()}, + }; } }; } diff --git a/dbms/src/Storages/System/StorageSystemCollations.cpp b/dbms/src/Storages/System/StorageSystemCollations.cpp index b75a2e70298..15609a39cd1 100644 --- a/dbms/src/Storages/System/StorageSystemCollations.cpp +++ b/dbms/src/Storages/System/StorageSystemCollations.cpp @@ -5,9 +5,9 @@ namespace DB { void StorageSystemCollations::fillData(MutableColumns & res_columns) const { - for (const auto & collation : Collator::getAvailableCollations()) + for (const auto & collation_name : Collator::getAvailableCollations()) { - res_columns[0]->insert(collation); + res_columns[0]->insert(collation_name); } } } diff --git a/dbms/src/Storages/System/StorageSystemCollations.h b/dbms/src/Storages/System/StorageSystemCollations.h index a10b19bb954..e00b932190a 100644 --- a/dbms/src/Storages/System/StorageSystemCollations.h +++ b/dbms/src/Storages/System/StorageSystemCollations.h @@ -1,26 +1,28 @@ #pragma once -#include +#include #include namespace DB { class StorageSystemCollations : public ext::shared_ptr_helper, - public IStorageSystemWithStringColumns + public IStorageSystemOneBlock { protected: void fillData(MutableColumns & res_columns) const override; public: - using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns; + using IStorageSystemOneBlock::IStorageSystemOneBlock; std::string getName() const override { return "SystemTableCollations"; } - static std::vector getColumnNames() + static NamesAndTypesList getNamesAndTypes() { - return {"name"}; + return { + {"name", std::make_shared()}, + }; } }; } diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index c2041ac044c..20c072327f2 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -12,39 +12,37 @@ namespace DB { namespace { - String getPropertiesAsString(const DataTypePtr data_type) + void setTypePropertries(const DataTypePtr data_type, MutableColumns & res_columns) { - std::vector properties; - if (data_type->isParametric()) - properties.push_back("parametric"); - if (data_type->haveSubtypes()) - properties.push_back("have_subtypes"); - if (data_type->cannotBeStoredInTables()) - properties.push_back("cannot_be_stored_in_tables"); - if (data_type->isComparable()) - properties.push_back("comparable"); - if (data_type->canBeComparedWithCollation()) - properties.push_back("can_be_compared_with_collation"); - if (data_type->canBeUsedAsVersion()) - properties.push_back("can_be_used_as_version"); - if (data_type->isSummable()) - properties.push_back("summable"); - if (data_type->canBeUsedInBitOperations()) - properties.push_back("can_be_used_in_bit_operations"); - if (data_type->canBeUsedInBooleanContext()) - properties.push_back("can_be_used_in_boolean_context"); - if (data_type->isValueRepresentedByNumber()) - properties.push_back("value_represented_by_number"); - if (data_type->isCategorial()) - properties.push_back("categorial"); - if (data_type->isNullable()) - properties.push_back("nullable"); - if (data_type->onlyNull()) - properties.push_back("only_null"); - if (data_type->canBeInsideNullable()) - properties.push_back("can_be_inside_nullable"); - return boost::algorithm::join(properties, ","); + res_columns[3]->insert(UInt64(data_type->isParametric())); + res_columns[4]->insert(UInt64(data_type->haveSubtypes())); + res_columns[5]->insert(UInt64(data_type->cannotBeStoredInTables())); + res_columns[6]->insert(UInt64(data_type->isComparable())); + res_columns[7]->insert(UInt64(data_type->canBeComparedWithCollation())); + res_columns[8]->insert(UInt64(data_type->canBeUsedAsVersion())); + res_columns[9]->insert(UInt64(data_type->isSummable())); + res_columns[10]->insert(UInt64(data_type->canBeUsedInBitOperations())); + res_columns[11]->insert(UInt64(data_type->canBeUsedInBooleanContext())); + res_columns[12]->insert(UInt64(data_type->isCategorial())); + res_columns[13]->insert(UInt64(data_type->isNullable())); + res_columns[14]->insert(UInt64(data_type->onlyNull())); + res_columns[15]->insert(UInt64(data_type->canBeInsideNullable())); } + + void setComplexTypeProperties(const String & name, MutableColumns & res_columns) + { + res_columns[3]->insert(UInt64(1)); //complex types are always parametric + if (name == "AggregateFunction") + res_columns[4]->insert(UInt64(0)); + else if (name == "Tuple") + res_columns[4]->insert(Null()); + else + res_columns[4]->insert(UInt64(1)); + + for (size_t i = 5; i < StorageSystemDataTypeFamilies::getNamesAndTypes().size(); ++i) + res_columns[i]->insert(Null()); + } + ASTPtr createFakeEnumCreationAst() { String fakename{"e"}; @@ -56,35 +54,46 @@ namespace clone->children.push_back(ast_func); return clone; } + + ASTPtr createFakeFixedStringAst() + { + ASTPtr result = std::make_shared(Field(UInt64(1))); + auto clone = result->clone(); + clone->children.clear(); + clone->children.push_back(result); + return clone; + } } void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns) const { const auto & factory = DataTypeFactory::instance(); - const auto & data_types = factory.getAllDataTypes(); - for (const auto & pair : data_types) + auto names = factory.getAllDataTypeNames(); + for (const auto & name : names) { - res_columns[0]->insert(pair.first); + res_columns[0]->insert(name); + res_columns[1]->insert(UInt64(factory.isCaseInsensitive(name))); + + if (factory.isAlias(name)) + res_columns[2]->insert(factory.aliasTo(name)); + else + res_columns[2]->insert(String("")); try { DataTypePtr type_ptr; - //special case with enum, because it has arguments but it's properties doesn't - //depend on arguments - if (boost::starts_with(pair.first, "Enum")) - { - type_ptr = factory.get(pair.first, createFakeEnumCreationAst()); - } + // hardcoded cases for simple parametric types + if (boost::starts_with(name, "Enum")) + type_ptr = factory.get(name, createFakeEnumCreationAst()); + else if (name == "FixedString" || name == "BINARY") + type_ptr = factory.get(name, createFakeFixedStringAst()); else - { - type_ptr = factory.get(pair.first); - } - - res_columns[1]->insert(getPropertiesAsString(type_ptr)); + type_ptr = factory.get(name); + setTypePropertries(type_ptr, res_columns); } catch (Exception & ex) { - res_columns[1]->insert(String{"depends_on_arguments"}); + setComplexTypeProperties(name, res_columns); } } } diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h index 38b769e6e1c..bc875f6130c 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -1,25 +1,46 @@ #pragma once -#include + +#include +#include +#include +#include #include namespace DB { class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper, - public IStorageSystemWithStringColumns + public IStorageSystemOneBlock { protected: void fillData(MutableColumns & res_columns) const override; public: - using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns; + using IStorageSystemOneBlock::IStorageSystemOneBlock; std::string getName() const override { return "SystemTableDataTypeFamilies"; } - static std::vector getColumnNames() + static NamesAndTypesList getNamesAndTypes() { - return {"name", "properties"}; + return { + {"name", std::make_shared()}, + {"case_insensivie", std::make_shared(std::make_shared())}, + {"alias_to", std::make_shared(std::make_shared())}, + {"parametric", std::make_shared(std::make_shared())}, + {"have_subtypes", std::make_shared(std::make_shared())}, + {"cannot_be_stored_in_tables", std::make_shared(std::make_shared())}, + {"comparable", std::make_shared(std::make_shared())}, + {"can_be_compared_with_collation", std::make_shared(std::make_shared())}, + {"can_be_used_as_version", std::make_shared(std::make_shared())}, + {"is_summable", std::make_shared(std::make_shared())}, + {"can_be_used_in_bit_operations", std::make_shared(std::make_shared())}, + {"can_be_used_in_boolean_context", std::make_shared(std::make_shared())}, + {"categorial", std::make_shared(std::make_shared())}, + {"nullable", std::make_shared(std::make_shared())}, + {"only_null", std::make_shared(std::make_shared())}, + {"can_be_inside_nullable", std::make_shared(std::make_shared())}, + }; } }; } diff --git a/dbms/src/Storages/System/StorageSystemFormats.cpp b/dbms/src/Storages/System/StorageSystemFormats.cpp index b029e354cc2..1032576b83e 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.cpp +++ b/dbms/src/Storages/System/StorageSystemFormats.cpp @@ -9,22 +9,11 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns) const for (const auto & pair : formats) { const auto & [name, creator_pair] = pair; - bool has_input_format = (creator_pair.first != nullptr); - bool has_output_format = (creator_pair.second != nullptr); + UInt64 has_input_format = UInt64(creator_pair.first != nullptr); + UInt64 has_output_format = UInt64(creator_pair.second != nullptr); res_columns[0]->insert(name); - std::string format_type; - if (has_input_format) - format_type = "input"; - - if (has_output_format) - { - if (!format_type.empty()) - format_type += "/output"; - else - format_type = "output"; - } - - res_columns[1]->insert(format_type); + res_columns[1]->insert(has_input_format); + res_columns[2]->insert(has_output_format); } } } diff --git a/dbms/src/Storages/System/StorageSystemFormats.h b/dbms/src/Storages/System/StorageSystemFormats.h index 35634bdbc21..91e3bcafbad 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.h +++ b/dbms/src/Storages/System/StorageSystemFormats.h @@ -1,26 +1,32 @@ #pragma once -#include +#include +#include +#include #include namespace DB { -class StorageSystemFormats : public ext::shared_ptr_helper, public IStorageSystemWithStringColumns +class StorageSystemFormats : public ext::shared_ptr_helper, public IStorageSystemOneBlock { protected: void fillData(MutableColumns & res_columns) const override; + public: - using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns; + using IStorageSystemOneBlock::IStorageSystemOneBlock; std::string getName() const override { return "SystemFormats"; } - static std::vector getColumnNames() + static NamesAndTypesList getNamesAndTypes() { - return {"name", "type"}; + return { + {"name", std::make_shared()}, + {"is_input", std::make_shared()}, + {"is_output", std::make_shared()}, + }; } - }; } diff --git a/dbms/src/Storages/System/StorageSystemTableFunctions.h b/dbms/src/Storages/System/StorageSystemTableFunctions.h index 096b3d8f4a4..c791ed216a8 100644 --- a/dbms/src/Storages/System/StorageSystemTableFunctions.h +++ b/dbms/src/Storages/System/StorageSystemTableFunctions.h @@ -1,26 +1,27 @@ #pragma once -#include +#include +#include #include namespace DB { class StorageSystemTableFunctions : public ext::shared_ptr_helper, - public IStorageSystemWithStringColumns + public IStorageSystemOneBlock { protected: void fillData(MutableColumns & res_columns) const override; public: - using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns; + using IStorageSystemOneBlock::IStorageSystemOneBlock; std::string getName() const override { return "SystemTableFunctions"; } - static std::vector getColumnNames() + static NamesAndTypesList getNamesAndTypes() { - return {"name"}; + return {{"name", std::make_shared()}}; } }; } From 24a9bebec9f434f88d50fa75843e52e4624dbf4b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Jul 2018 19:13:35 +0300 Subject: [PATCH 02/17] CLICKHOUSE-3772: Remove unused import --- dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index 20c072327f2..5d731803d4b 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -6,8 +6,6 @@ #include #include -#include - namespace DB { namespace From f2451210d021df35a5c624e1d4f1126f309f6b13 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Jul 2018 19:52:11 +0300 Subject: [PATCH 03/17] CLICKHOUSE-3772: Add size column --- .../System/StorageSystemDataTypeFamilies.cpp | 51 ++++++++++++------- .../System/StorageSystemDataTypeFamilies.h | 1 + 2 files changed, 34 insertions(+), 18 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index 5d731803d4b..a97ac696977 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -12,32 +12,33 @@ namespace { void setTypePropertries(const DataTypePtr data_type, MutableColumns & res_columns) { - res_columns[3]->insert(UInt64(data_type->isParametric())); - res_columns[4]->insert(UInt64(data_type->haveSubtypes())); - res_columns[5]->insert(UInt64(data_type->cannotBeStoredInTables())); - res_columns[6]->insert(UInt64(data_type->isComparable())); - res_columns[7]->insert(UInt64(data_type->canBeComparedWithCollation())); - res_columns[8]->insert(UInt64(data_type->canBeUsedAsVersion())); - res_columns[9]->insert(UInt64(data_type->isSummable())); - res_columns[10]->insert(UInt64(data_type->canBeUsedInBitOperations())); - res_columns[11]->insert(UInt64(data_type->canBeUsedInBooleanContext())); - res_columns[12]->insert(UInt64(data_type->isCategorial())); - res_columns[13]->insert(UInt64(data_type->isNullable())); - res_columns[14]->insert(UInt64(data_type->onlyNull())); - res_columns[15]->insert(UInt64(data_type->canBeInsideNullable())); + res_columns[4]->insert(UInt64(data_type->isParametric())); + res_columns[5]->insert(UInt64(data_type->haveSubtypes())); + res_columns[6]->insert(UInt64(data_type->cannotBeStoredInTables())); + res_columns[7]->insert(UInt64(data_type->isComparable())); + res_columns[8]->insert(UInt64(data_type->canBeComparedWithCollation())); + res_columns[9]->insert(UInt64(data_type->canBeUsedAsVersion())); + res_columns[10]->insert(UInt64(data_type->isSummable())); + res_columns[11]->insert(UInt64(data_type->canBeUsedInBitOperations())); + res_columns[12]->insert(UInt64(data_type->canBeUsedInBooleanContext())); + res_columns[13]->insert(UInt64(data_type->isCategorial())); + res_columns[14]->insert(UInt64(data_type->isNullable())); + res_columns[15]->insert(UInt64(data_type->onlyNull())); + res_columns[16]->insert(UInt64(data_type->canBeInsideNullable())); } void setComplexTypeProperties(const String & name, MutableColumns & res_columns) { - res_columns[3]->insert(UInt64(1)); //complex types are always parametric + res_columns[3]->insert(Null()); + res_columns[4]->insert(UInt64(1)); //complex types are always parametric if (name == "AggregateFunction") - res_columns[4]->insert(UInt64(0)); + res_columns[5]->insert(UInt64(0)); else if (name == "Tuple") - res_columns[4]->insert(Null()); + res_columns[5]->insert(Null()); else - res_columns[4]->insert(UInt64(1)); + res_columns[5]->insert(UInt64(1)); - for (size_t i = 5; i < StorageSystemDataTypeFamilies::getNamesAndTypes().size(); ++i) + for (size_t i = 6; i < StorageSystemDataTypeFamilies::getNamesAndTypes().size(); ++i) res_columns[i]->insert(Null()); } @@ -80,13 +81,27 @@ void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns) const try { DataTypePtr type_ptr; + + Field size = Null(); // hardcoded cases for simple parametric types if (boost::starts_with(name, "Enum")) + { type_ptr = factory.get(name, createFakeEnumCreationAst()); + size = type_ptr->getMaximumSizeOfValueInMemory(); + } else if (name == "FixedString" || name == "BINARY") + { type_ptr = factory.get(name, createFakeFixedStringAst()); + } else + { type_ptr = factory.get(name); + if (type_ptr->haveMaximumSizeOfValue()) + size = type_ptr->getMaximumSizeOfValueInMemory(); + } + + res_columns[3]->insert(size); + setTypePropertries(type_ptr, res_columns); } catch (Exception & ex) diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h index bc875f6130c..da2b338d53b 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -27,6 +27,7 @@ public: {"name", std::make_shared()}, {"case_insensivie", std::make_shared(std::make_shared())}, {"alias_to", std::make_shared(std::make_shared())}, + {"size_in_bytes", std::make_shared(std::make_shared())}, {"parametric", std::make_shared(std::make_shared())}, {"have_subtypes", std::make_shared(std::make_shared())}, {"cannot_be_stored_in_tables", std::make_shared(std::make_shared())}, From 220f4d6c43fd0f3d23083541c60778c7668de3c4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Jul 2018 20:12:29 +0300 Subject: [PATCH 04/17] CLICKHOUSE-3772: Fix strange initialization --- dbms/src/Storages/System/StorageSystemFormats.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemFormats.cpp b/dbms/src/Storages/System/StorageSystemFormats.cpp index 1032576b83e..63475a343f9 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.cpp +++ b/dbms/src/Storages/System/StorageSystemFormats.cpp @@ -9,8 +9,8 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns) const for (const auto & pair : formats) { const auto & [name, creator_pair] = pair; - UInt64 has_input_format = UInt64(creator_pair.first != nullptr); - UInt64 has_output_format = UInt64(creator_pair.second != nullptr); + UInt64 has_input_format(creator_pair.first != nullptr); + UInt64 has_output_format(creator_pair.second != nullptr); res_columns[0]->insert(name); res_columns[1]->insert(has_input_format); res_columns[2]->insert(has_output_format); From 4835b4e16e47fec73c8c5b1f2e44bf61ccbb1579 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Jul 2018 20:46:18 +0300 Subject: [PATCH 05/17] CLICKHOUSE-3772: Fix brackets --- dbms/src/DataTypes/DataTypeFactory.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 72f34b23836..7379ac85b64 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -99,7 +99,8 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat ErrorCodes::LOGICAL_ERROR); } -void DataTypeFactory::registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) { +void DataTypeFactory::registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) +{ String real_type_dict_name; if (data_types.count(real_name)) real_type_dict_name = real_name; @@ -161,7 +162,8 @@ const String & DataTypeFactory::aliasTo(const String & name) const } -bool DataTypeFactory::isAlias(const String & name) const { +bool DataTypeFactory::isAlias(const String & name) const +{ return aliases.count(name) || case_insensitive_aliases.count(name); } From 5b82080185ff866736bcd3852ed02ac7832630de Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 24 Jul 2018 00:19:21 +0300 Subject: [PATCH 06/17] Update DataTypeFactory.cpp --- dbms/src/DataTypes/DataTypeFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 7379ac85b64..74e28cf9014 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -118,7 +118,7 @@ void DataTypeFactory::registerAlias(const String & alias_name, const String & re if(!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second) throw Exception("DataTypeFactory: case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - if(!aliases.emplace(alias_name, real_type_dict_name).second) + if (!aliases.emplace(alias_name, real_type_dict_name).second) throw Exception("DataTypeFactory: alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } From 159b377860131656e53d3823ecfa148719132e6d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 24 Jul 2018 00:20:56 +0300 Subject: [PATCH 07/17] Update DataTypeFactory.cpp --- dbms/src/DataTypes/DataTypeFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 74e28cf9014..5a067a6871e 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -139,7 +139,7 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator std::vector DataTypeFactory::getAllDataTypeNames() const { std::vector result; - auto getter = [] (const auto& pair) { return pair.first; }; + auto getter = [] (const auto & pair) { return pair.first; }; std::transform(data_types.begin(), data_types.end(), std::back_inserter(result), getter); std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter); return result; From 670353895bb8142404de5bf0fe2fa8cb40e804de Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Jul 2018 17:28:56 +0300 Subject: [PATCH 08/17] CLICKHOUSE-3772: Change base class of almost all StorageSystem* tables and fix remarks --- .../AggregateFunctionFactory.cpp | 4 +- .../AggregateFunctionNull.cpp | 2 + .../IAggregateFunctionCombinator.h | 2 + dbms/src/DataTypes/DataTypeFactory.cpp | 6 +- .../Storages/System/IStorageSystemOneBlock.h | 8 +- ...rageSystemAggregateFunctionCombinators.cpp | 18 +++- ...torageSystemAggregateFunctionCombinators.h | 12 +-- .../StorageSystemAsynchronousMetrics.cpp | 37 ++----- .../System/StorageSystemAsynchronousMetrics.h | 20 ++-- .../System/StorageSystemBuildOptions.cpp | 33 ++----- .../System/StorageSystemBuildOptions.h | 30 +++--- .../Storages/System/StorageSystemClusters.cpp | 59 ++++------- .../Storages/System/StorageSystemClusters.h | 22 ++--- .../System/StorageSystemCollations.cpp | 13 ++- .../Storages/System/StorageSystemCollations.h | 18 ++-- .../Storages/System/StorageSystemColumns.cpp | 28 ++---- .../Storages/System/StorageSystemColumns.h | 18 +--- .../System/StorageSystemDataTypeFamilies.cpp | 98 ++----------------- .../System/StorageSystemDataTypeFamilies.h | 34 ++----- .../System/StorageSystemDatabases.cpp | 30 +----- .../Storages/System/StorageSystemDatabases.h | 26 +++-- .../System/StorageSystemDictionaries.cpp | 33 +------ .../System/StorageSystemDictionaries.h | 24 ++--- .../Storages/System/StorageSystemEvents.cpp | 31 +----- .../src/Storages/System/StorageSystemEvents.h | 23 ++--- .../Storages/System/StorageSystemFormats.cpp | 13 ++- .../Storages/System/StorageSystemFormats.h | 13 +-- .../System/StorageSystemFunctions.cpp | 35 ++----- .../Storages/System/StorageSystemFunctions.h | 20 ++-- .../Storages/System/StorageSystemGraphite.cpp | 22 +---- .../Storages/System/StorageSystemGraphite.h | 22 ++--- .../Storages/System/StorageSystemMacros.cpp | 36 ++----- .../src/Storages/System/StorageSystemMacros.h | 21 ++-- .../Storages/System/StorageSystemMerges.cpp | 61 ++++-------- .../src/Storages/System/StorageSystemMerges.h | 23 ++--- .../Storages/System/StorageSystemMetrics.cpp | 31 ++---- .../Storages/System/StorageSystemMetrics.h | 20 ++-- .../Storages/System/StorageSystemModels.cpp | 26 +---- .../src/Storages/System/StorageSystemModels.h | 20 ++-- .../System/StorageSystemMutations.cpp | 28 ++---- .../Storages/System/StorageSystemMutations.h | 20 ++-- .../System/StorageSystemProcesses.cpp | 86 +++++++--------- .../Storages/System/StorageSystemProcesses.h | 20 ++-- .../System/StorageSystemReplicationQueue.cpp | 27 ++--- .../System/StorageSystemReplicationQueue.h | 20 ++-- .../Storages/System/StorageSystemSettings.cpp | 42 +++----- .../Storages/System/StorageSystemSettings.h | 20 ++-- .../System/StorageSystemTableFunctions.cpp | 9 +- .../System/StorageSystemTableFunctions.h | 14 +-- .../System/StorageSystemZooKeeper.cpp | 26 +---- .../Storages/System/StorageSystemZooKeeper.h | 20 ++-- 51 files changed, 392 insertions(+), 932 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index eca854a031b..366aef1f829 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -103,8 +103,8 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name)) { - if (combinator->getName() == "Null") - throw Exception("Aggregate function combinator 'Null' is only for internal usage", ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); + if (combinator->isForInternalUsageOnly()) + throw Exception("Aggregate function combinator '" + combinator->getName() + "' is only for internal usage", ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); String nested_name = name.substr(0, name.size() - combinator->getName().size()); DataTypes nested_types = combinator->transformArguments(argument_types); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp b/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp index 46a46a2370a..6ce7d94d970 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -18,6 +18,8 @@ class AggregateFunctionCombinatorNull final : public IAggregateFunctionCombinato public: String getName() const override { return "Null"; } + bool isForInternalUsageOnly() const override { return true; } + DataTypes transformArguments(const DataTypes & arguments) const override { size_t size = arguments.size(); diff --git a/dbms/src/AggregateFunctions/IAggregateFunctionCombinator.h b/dbms/src/AggregateFunctions/IAggregateFunctionCombinator.h index ba28026b1cd..0ac9a3d41cd 100644 --- a/dbms/src/AggregateFunctions/IAggregateFunctionCombinator.h +++ b/dbms/src/AggregateFunctions/IAggregateFunctionCombinator.h @@ -32,6 +32,8 @@ class IAggregateFunctionCombinator public: virtual String getName() const = 0; + virtual bool isForInternalUsageOnly() const { return false; } + /** From the arguments for combined function (ex: UInt64, UInt8 for sumIf), * get the arguments for nested function (ex: UInt64 for sum). * If arguments are not suitable for combined function, throw an exception. diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 7379ac85b64..87599a90370 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -115,10 +115,10 @@ void DataTypeFactory::registerAlias(const String & alias_name, const String & re throw Exception("DataTypeFactory: the alias name " + alias_name + " is already registered as datatype", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive) - if(!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second) + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second) throw Exception("DataTypeFactory: case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - if(!aliases.emplace(alias_name, real_type_dict_name).second) + if (!aliases.emplace(alias_name, real_type_dict_name).second) throw Exception("DataTypeFactory: alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } @@ -139,7 +139,7 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator std::vector DataTypeFactory::getAllDataTypeNames() const { std::vector result; - auto getter = [] (const auto& pair) { return pair.first; }; + auto getter = [] (const auto & pair) { return pair.first; }; std::transform(data_types.begin(), data_types.end(), std::back_inserter(result), getter); std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter); return result; diff --git a/dbms/src/Storages/System/IStorageSystemOneBlock.h b/dbms/src/Storages/System/IStorageSystemOneBlock.h index ad50b43b975..96286f56eee 100644 --- a/dbms/src/Storages/System/IStorageSystemOneBlock.h +++ b/dbms/src/Storages/System/IStorageSystemOneBlock.h @@ -17,7 +17,7 @@ template class IStorageSystemOneBlock : public IStorage { protected: - virtual void fillData(MutableColumns & res_columns) const = 0; + virtual void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const = 0; public: IStorageSystemOneBlock(const String & name_) : name(name_) @@ -31,8 +31,8 @@ public: } BlockInputStreams read(const Names & column_names, - const SelectQueryInfo & /*query_info*/, - const Context & /*context*/, + const SelectQueryInfo & query_info, + const Context & context, QueryProcessingStage::Enum & processed_stage, size_t /*max_block_size*/, unsigned /*num_streams*/) override @@ -42,7 +42,7 @@ public: Block sample_block = getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); - fillData(res_columns); + fillData(res_columns, context, query_info); return BlockInputStreams(1, std::make_shared(sample_block.cloneWithColumns(std::move(res_columns)))); } diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp index 2f23d66aac2..3fb91c6ce8a 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp @@ -3,16 +3,26 @@ namespace DB { -void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_columns) const + +NamesAndTypesList StorageSystemAggregateFunctionCombinators::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"is_internal", std::make_shared()}, + }; +} + +void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { const auto & combinators = AggregateFunctionCombinatorFactory::instance().getAllAggregateFunctionCombinators(); for (const auto & pair : combinators) { res_columns[0]->insert(pair.first); - if (pair.first != "Null") - res_columns[1]->insert(UInt64(0)); - else + if (pair.second->isForInternalUsageOnly()) res_columns[1]->insert(UInt64(1)); + else + res_columns[1]->insert(UInt64(0)); } } + } diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h index 56aaabad463..1d7226eda8b 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h @@ -10,22 +10,16 @@ class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper< public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns) const override; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; -public: using IStorageSystemOneBlock::IStorageSystemOneBlock; +public: std::string getName() const override { return "SystemAggregateFunctionCombinators"; } - static NamesAndTypesList getNamesAndTypes() - { - return { - {"name", std::make_shared()}, - {"is_internal", std::make_shared()}, - }; - } + static NamesAndTypesList getNamesAndTypes(); }; } diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index bc2f76379e9..f6c8e1c9953 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -1,51 +1,32 @@ -#include - #include -#include -#include -#include -#include -#include +#include namespace DB { - -StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_) - : name(name_), - async_metrics(async_metrics_) +NamesAndTypesList StorageSystemAsynchronousMetrics::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { {"metric", std::make_shared()}, {"value", std::make_shared()}, - })); + }; } -BlockInputStreams StorageSystemAsynchronousMetrics::read( - const Names & column_names, - const SelectQueryInfo &, - const Context &, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +StorageSystemAsynchronousMetrics::StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_) + : IStorageSystemOneBlock(name_), async_metrics(async_metrics_) { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); +} +void StorageSystemAsynchronousMetrics::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +{ auto async_metrics_values = async_metrics.getValues(); - for (const auto & name_value : async_metrics_values) { res_columns[0]->insert(name_value.first); res_columns[1]->insert(name_value.second); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h index 60e50096143..e0e45e25a50 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -1,8 +1,10 @@ #pragma once -#include -#include +#include +#include +#include +#include namespace DB { @@ -13,26 +15,20 @@ class Context; /** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics. */ -class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, public IStorage +class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemAsynchronousMetrics"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; + static NamesAndTypesList getNamesAndTypes(); private: - const std::string name; const AsynchronousMetrics & async_metrics; protected: StorageSystemAsynchronousMetrics(const std::string & name_, const AsynchronousMetrics & async_metrics_); + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.cpp b/dbms/src/Storages/System/StorageSystemBuildOptions.cpp index e62e6e9bbfd..621854f1a5f 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.cpp +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.cpp @@ -1,7 +1,3 @@ -#include -#include -#include -#include #include #include #include @@ -9,38 +5,21 @@ namespace DB { - -StorageSystemBuildOptions::StorageSystemBuildOptions(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemBuildOptions::getNamesAndTypes() { - setColumns(ColumnsDescription({ - { "name", std::make_shared() }, - { "value", std::make_shared() }, - })); + return { + {"name", std::make_shared()}, + {"value", std::make_shared()}, + }; } - -BlockInputStreams StorageSystemBuildOptions::read( - const Names & column_names, - const SelectQueryInfo &, - const Context &, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemBuildOptions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (auto it = auto_config_build; *it; it += 2) { res_columns[0]->insert(String(it[0])); res_columns[1]->insert(String(it[1])); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.h b/dbms/src/Storages/System/StorageSystemBuildOptions.h index d772b255383..a8909addf85 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.h +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.h @@ -1,7 +1,8 @@ #pragma once +#include #include -#include +#include namespace DB @@ -12,25 +13,18 @@ class Context; /** System table "build_options" with many params used for clickhouse building */ -class StorageSystemBuildOptions : public ext::shared_ptr_helper, public IStorage +class StorageSystemBuildOptions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { -public: - std::string getName() const override { return "SystemBuildOptions"; } - std::string getTableName() const override { return name; } - - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; - protected: - StorageSystemBuildOptions(const std::string & name_); + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; + + using IStorageSystemOneBlock::IStorageSystemOneBlock; + +public: + + std::string getName() const override { return "SystemBuildOptions"; } + + static NamesAndTypesList getNamesAndTypes(); }; } diff --git a/dbms/src/Storages/System/StorageSystemClusters.cpp b/dbms/src/Storages/System/StorageSystemClusters.cpp index fb5c4e41b82..693d127b7c7 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.cpp +++ b/dbms/src/Storages/System/StorageSystemClusters.cpp @@ -1,51 +1,29 @@ -#include #include -#include -#include -#include -#include -#include -#include #include +#include +#include namespace DB { - - -StorageSystemClusters::StorageSystemClusters(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemClusters::getNamesAndTypes() { - setColumns(ColumnsDescription({ - { "cluster", std::make_shared() }, - { "shard_num", std::make_shared() }, - { "shard_weight", std::make_shared() }, - { "replica_num", std::make_shared() }, - { "host_name", std::make_shared() }, - { "host_address", std::make_shared() }, - { "port", std::make_shared() }, - { "is_local", std::make_shared() }, - { "user", std::make_shared() }, - { "default_database", std::make_shared() }, - })); + return { + {"cluster", std::make_shared()}, + {"shard_num", std::make_shared()}, + {"shard_weight", std::make_shared()}, + {"replica_num", std::make_shared()}, + {"host_name", std::make_shared()}, + {"host_address", std::make_shared()}, + {"port", std::make_shared()}, + {"is_local", std::make_shared()}, + {"user", std::make_shared()}, + {"default_database", std::make_shared()}, + }; } - -BlockInputStreams StorageSystemClusters::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemClusters::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - - auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info, - const Cluster::Address & address) - { + auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info, const Cluster::Address & address) { size_t i = 0; res_columns[i++]->insert(cluster_name); res_columns[i++]->insert(static_cast(shard_info.shard_num)); @@ -85,8 +63,5 @@ BlockInputStreams StorageSystemClusters::read( } } } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemClusters.h b/dbms/src/Storages/System/StorageSystemClusters.h index 1e36269ded2..dde9e53b626 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.h +++ b/dbms/src/Storages/System/StorageSystemClusters.h @@ -1,7 +1,9 @@ #pragma once +#include +#include #include -#include +#include namespace DB @@ -13,25 +15,17 @@ class Context; * that allows to obtain information about available clusters * (which may be specified in Distributed tables). */ -class StorageSystemClusters : public ext::shared_ptr_helper, public IStorage +class StorageSystemClusters : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemClusters"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemClusters(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemCollations.cpp b/dbms/src/Storages/System/StorageSystemCollations.cpp index 15609a39cd1..f2a7f5e8184 100644 --- a/dbms/src/Storages/System/StorageSystemCollations.cpp +++ b/dbms/src/Storages/System/StorageSystemCollations.cpp @@ -3,11 +3,18 @@ namespace DB { -void StorageSystemCollations::fillData(MutableColumns & res_columns) const + +NamesAndTypesList StorageSystemCollations::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + }; +} + +void StorageSystemCollations::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { for (const auto & collation_name : Collator::getAvailableCollations()) - { res_columns[0]->insert(collation_name); - } } + } diff --git a/dbms/src/Storages/System/StorageSystemCollations.h b/dbms/src/Storages/System/StorageSystemCollations.h index e00b932190a..f8b7b6ee3af 100644 --- a/dbms/src/Storages/System/StorageSystemCollations.h +++ b/dbms/src/Storages/System/StorageSystemCollations.h @@ -4,25 +4,19 @@ namespace DB { + class StorageSystemCollations : public ext::shared_ptr_helper, public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns) const override; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; -public: using IStorageSystemOneBlock::IStorageSystemOneBlock; +public: - std::string getName() const override - { - return "SystemTableCollations"; - } + std::string getName() const override { return "SystemTableCollations"; } - static NamesAndTypesList getNamesAndTypes() - { - return { - {"name", std::make_shared()}, - }; - } + static NamesAndTypesList getNamesAndTypes(); }; + } diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index d42d8a80394..171ae302b83 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -15,10 +15,8 @@ namespace DB { -StorageSystemColumns::StorageSystemColumns(const std::string & name_) - : name(name_) -{ - setColumns(ColumnsDescription({ +NamesAndTypesList StorageSystemColumns::getNamesAndTypes() { + return { { "database", std::make_shared() }, { "table", std::make_shared() }, { "name", std::make_shared() }, @@ -28,21 +26,11 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "data_compressed_bytes", std::make_shared() }, { "data_uncompressed_bytes", std::make_shared() }, { "marks_bytes", std::make_shared() }, - })); + }; } - -BlockInputStreams StorageSystemColumns::read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemColumns::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - Block block_to_filter; std::map, StoragePtr> storages; @@ -60,7 +48,7 @@ BlockInputStreams StorageSystemColumns::read( VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); if (!block_to_filter.rows()) - return BlockInputStreams(); + return; ColumnPtr database_column = block_to_filter.getByName("database").column; size_t rows = database_column->size(); @@ -98,14 +86,12 @@ BlockInputStreams StorageSystemColumns::read( VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); if (!block_to_filter.rows()) - return BlockInputStreams(); + return; ColumnPtr filtered_database_column = block_to_filter.getByName("database").column; ColumnPtr filtered_table_column = block_to_filter.getByName("table").column; /// We compose the result. - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - size_t rows = filtered_database_column->size(); for (size_t i = 0; i < rows; ++i) { @@ -193,8 +179,6 @@ BlockInputStreams StorageSystemColumns::read( } } } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } } diff --git a/dbms/src/Storages/System/StorageSystemColumns.h b/dbms/src/Storages/System/StorageSystemColumns.h index ba187f7306f..dc1afc3f71a 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.h +++ b/dbms/src/Storages/System/StorageSystemColumns.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -11,25 +11,17 @@ class Context; /** Implements system table 'columns', that allows to get information about columns for every table. */ -class StorageSystemColumns : public ext::shared_ptr_helper, public IStorage +class StorageSystemColumns : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemColumns"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemColumns(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; -private: - const std::string name; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index a97ac696977..7f154e9f17b 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -1,70 +1,20 @@ #include #include -#include -#include #include -#include -#include namespace DB { -namespace + +NamesAndTypesList StorageSystemDataTypeFamilies::getNamesAndTypes() { - void setTypePropertries(const DataTypePtr data_type, MutableColumns & res_columns) - { - res_columns[4]->insert(UInt64(data_type->isParametric())); - res_columns[5]->insert(UInt64(data_type->haveSubtypes())); - res_columns[6]->insert(UInt64(data_type->cannotBeStoredInTables())); - res_columns[7]->insert(UInt64(data_type->isComparable())); - res_columns[8]->insert(UInt64(data_type->canBeComparedWithCollation())); - res_columns[9]->insert(UInt64(data_type->canBeUsedAsVersion())); - res_columns[10]->insert(UInt64(data_type->isSummable())); - res_columns[11]->insert(UInt64(data_type->canBeUsedInBitOperations())); - res_columns[12]->insert(UInt64(data_type->canBeUsedInBooleanContext())); - res_columns[13]->insert(UInt64(data_type->isCategorial())); - res_columns[14]->insert(UInt64(data_type->isNullable())); - res_columns[15]->insert(UInt64(data_type->onlyNull())); - res_columns[16]->insert(UInt64(data_type->canBeInsideNullable())); - } - - void setComplexTypeProperties(const String & name, MutableColumns & res_columns) - { - res_columns[3]->insert(Null()); - res_columns[4]->insert(UInt64(1)); //complex types are always parametric - if (name == "AggregateFunction") - res_columns[5]->insert(UInt64(0)); - else if (name == "Tuple") - res_columns[5]->insert(Null()); - else - res_columns[5]->insert(UInt64(1)); - - for (size_t i = 6; i < StorageSystemDataTypeFamilies::getNamesAndTypes().size(); ++i) - res_columns[i]->insert(Null()); - } - - ASTPtr createFakeEnumCreationAst() - { - String fakename{"e"}; - ASTPtr name = std::make_shared(Field(fakename.c_str(), fakename.size())); - ASTPtr value = std::make_shared(Field(UInt64(1))); - ASTPtr ast_func = makeASTFunction("equals", name, value); - ASTPtr clone = ast_func->clone(); - clone->children.clear(); - clone->children.push_back(ast_func); - return clone; - } - - ASTPtr createFakeFixedStringAst() - { - ASTPtr result = std::make_shared(Field(UInt64(1))); - auto clone = result->clone(); - clone->children.clear(); - clone->children.push_back(result); - return clone; - } + return { + {"name", std::make_shared()}, + {"case_insensivie", std::make_shared(std::make_shared())}, + {"alias_to", std::make_shared(std::make_shared())}, + }; } -void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns) const +void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { const auto & factory = DataTypeFactory::instance(); auto names = factory.getAllDataTypeNames(); @@ -77,37 +27,7 @@ void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns) const res_columns[2]->insert(factory.aliasTo(name)); else res_columns[2]->insert(String("")); - - try - { - DataTypePtr type_ptr; - - Field size = Null(); - // hardcoded cases for simple parametric types - if (boost::starts_with(name, "Enum")) - { - type_ptr = factory.get(name, createFakeEnumCreationAst()); - size = type_ptr->getMaximumSizeOfValueInMemory(); - } - else if (name == "FixedString" || name == "BINARY") - { - type_ptr = factory.get(name, createFakeFixedStringAst()); - } - else - { - type_ptr = factory.get(name); - if (type_ptr->haveMaximumSizeOfValue()) - size = type_ptr->getMaximumSizeOfValueInMemory(); - } - - res_columns[3]->insert(size); - - setTypePropertries(type_ptr, res_columns); - } - catch (Exception & ex) - { - setComplexTypeProperties(name, res_columns); - } } } + } diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h index da2b338d53b..3cc1c50d426 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -7,41 +7,19 @@ #include namespace DB { + class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper, public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns) const override; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; -public: using IStorageSystemOneBlock::IStorageSystemOneBlock; - std::string getName() const override - { - return "SystemTableDataTypeFamilies"; - } +public: + std::string getName() const override { return "SystemTableDataTypeFamilies"; } - static NamesAndTypesList getNamesAndTypes() - { - return { - {"name", std::make_shared()}, - {"case_insensivie", std::make_shared(std::make_shared())}, - {"alias_to", std::make_shared(std::make_shared())}, - {"size_in_bytes", std::make_shared(std::make_shared())}, - {"parametric", std::make_shared(std::make_shared())}, - {"have_subtypes", std::make_shared(std::make_shared())}, - {"cannot_be_stored_in_tables", std::make_shared(std::make_shared())}, - {"comparable", std::make_shared(std::make_shared())}, - {"can_be_compared_with_collation", std::make_shared(std::make_shared())}, - {"can_be_used_as_version", std::make_shared(std::make_shared())}, - {"is_summable", std::make_shared(std::make_shared())}, - {"can_be_used_in_bit_operations", std::make_shared(std::make_shared())}, - {"can_be_used_in_boolean_context", std::make_shared(std::make_shared())}, - {"categorial", std::make_shared(std::make_shared())}, - {"nullable", std::make_shared(std::make_shared())}, - {"only_null", std::make_shared(std::make_shared())}, - {"can_be_inside_nullable", std::make_shared(std::make_shared())}, - }; - } + static NamesAndTypesList getNamesAndTypes(); }; + } diff --git a/dbms/src/Storages/System/StorageSystemDatabases.cpp b/dbms/src/Storages/System/StorageSystemDatabases.cpp index 49c78688616..7145cb36178 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.cpp +++ b/dbms/src/Storages/System/StorageSystemDatabases.cpp @@ -1,40 +1,23 @@ -#include -#include -#include #include -#include #include +#include namespace DB { - -StorageSystemDatabases::StorageSystemDatabases(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemDatabases::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { {"name", std::make_shared()}, {"engine", std::make_shared()}, {"data_path", std::make_shared()}, {"metadata_path", std::make_shared()}, - })); + }; } - -BlockInputStreams StorageSystemDatabases::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - auto databases = context.getDatabases(); for (const auto & database : databases) { @@ -43,9 +26,6 @@ BlockInputStreams StorageSystemDatabases::read( res_columns[2]->insert(database.second->getDataPath()); res_columns[3]->insert(database.second->getMetadataPath()); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemDatabases.h b/dbms/src/Storages/System/StorageSystemDatabases.h index 621e490963a..26052c726c5 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.h +++ b/dbms/src/Storages/System/StorageSystemDatabases.h @@ -1,7 +1,8 @@ #pragma once +#include +#include #include -#include namespace DB @@ -12,25 +13,20 @@ class Context; /** Implements `databases` system table, which allows you to get information about all databases. */ -class StorageSystemDatabases : public ext::shared_ptr_helper, public IStorage +class StorageSystemDatabases : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: - std::string getName() const override { return "SystemDatabases"; } - std::string getTableName() const override { return name; } + std::string getName() const override + { + return "SystemDatabases"; + } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemDatabases(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index c57c1c7f459..ae52af89689 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -1,12 +1,4 @@ #include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -18,10 +10,9 @@ namespace DB { -StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name) - : name{name} +NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { { "name", std::make_shared() }, { "origin", std::make_shared() }, { "type", std::make_shared() }, @@ -36,27 +27,13 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name) { "creation_time", std::make_shared() }, { "source", std::make_shared() }, { "last_exception", std::make_shared() }, - })); + }; } - -BlockInputStreams StorageSystemDictionaries::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t, - const unsigned) -{ - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - +void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { const auto & external_dictionaries = context.getExternalDictionaries(); auto objects_map = external_dictionaries.getObjectsMap(); const auto & dictionaries = objects_map.get(); - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (const auto & dict_info : dictionaries) { size_t i = 0; @@ -102,8 +79,6 @@ BlockInputStreams StorageSystemDictionaries::read( else res_columns[i++]->insertDefault(); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } } diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.h b/dbms/src/Storages/System/StorageSystemDictionaries.h index 57ac9b0b6eb..57512692809 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.h +++ b/dbms/src/Storages/System/StorageSystemDictionaries.h @@ -1,7 +1,11 @@ #pragma once +#include +#include +#include +#include #include -#include +#include namespace DB @@ -10,25 +14,17 @@ namespace DB class Context; -class StorageSystemDictionaries : public ext::shared_ptr_helper, public IStorage +class StorageSystemDictionaries : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemDictionaries"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemDictionaries(const std::string & name); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemEvents.cpp b/dbms/src/Storages/System/StorageSystemEvents.cpp index 1dc49ad37b2..02b7c776086 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.cpp +++ b/dbms/src/Storages/System/StorageSystemEvents.cpp @@ -1,39 +1,19 @@ #include -#include -#include -#include -#include #include - namespace DB { - -StorageSystemEvents::StorageSystemEvents(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemEvents::getNamesAndTypes() { - setColumns(ColumnsDescription( - { + return { {"event", std::make_shared()}, {"value", std::make_shared()}, - })); + }; } - -BlockInputStreams StorageSystemEvents::read( - const Names & column_names, - const SelectQueryInfo &, - const Context &, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemEvents::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { UInt64 value = ProfileEvents::counters[i]; @@ -44,9 +24,6 @@ BlockInputStreams StorageSystemEvents::read( res_columns[1]->insert(value); } } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemEvents.h b/dbms/src/Storages/System/StorageSystemEvents.h index b987151e400..3c3469c3656 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.h +++ b/dbms/src/Storages/System/StorageSystemEvents.h @@ -1,8 +1,9 @@ #pragma once +#include +#include #include -#include - +#include namespace DB { @@ -12,25 +13,17 @@ class Context; /** Implements `events` system table, which allows you to obtain information for profiling. */ -class StorageSystemEvents : public ext::shared_ptr_helper, public IStorage +class StorageSystemEvents : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemEvents"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemEvents(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemFormats.cpp b/dbms/src/Storages/System/StorageSystemFormats.cpp index 63475a343f9..13df91b6fc4 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.cpp +++ b/dbms/src/Storages/System/StorageSystemFormats.cpp @@ -3,7 +3,17 @@ namespace DB { -void StorageSystemFormats::fillData(MutableColumns & res_columns) const + +NamesAndTypesList StorageSystemFormats::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"is_input", std::make_shared()}, + {"is_output", std::make_shared()}, + }; +} + +void StorageSystemFormats::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { const auto & formats = FormatFactory::instance().getAllFormats(); for (const auto & pair : formats) @@ -16,4 +26,5 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns) const res_columns[2]->insert(has_output_format); } } + } diff --git a/dbms/src/Storages/System/StorageSystemFormats.h b/dbms/src/Storages/System/StorageSystemFormats.h index 91e3bcafbad..9b040ada1ca 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.h +++ b/dbms/src/Storages/System/StorageSystemFormats.h @@ -10,23 +10,16 @@ namespace DB class StorageSystemFormats : public ext::shared_ptr_helper, public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns) const override; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; -public: using IStorageSystemOneBlock::IStorageSystemOneBlock; +public: std::string getName() const override { return "SystemFormats"; } - static NamesAndTypesList getNamesAndTypes() - { - return { - {"name", std::make_shared()}, - {"is_input", std::make_shared()}, - {"is_output", std::make_shared()}, - }; - } + static NamesAndTypesList getNamesAndTypes(); }; } diff --git a/dbms/src/Storages/System/StorageSystemFunctions.cpp b/dbms/src/Storages/System/StorageSystemFunctions.cpp index 909bf1d8089..0c22a022cb0 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.cpp +++ b/dbms/src/Storages/System/StorageSystemFunctions.cpp @@ -1,41 +1,28 @@ -#include -#include -#include #include #include #include +#include #include #include -#include +#include +#include #include +#include namespace DB { -StorageSystemFunctions::StorageSystemFunctions(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemFunctions::getNamesAndTypes() { - setColumns(ColumnsDescription({ - { "name", std::make_shared() }, - { "is_aggregate", std::make_shared() }, - })); + return { + {"name", std::make_shared()}, + {"is_aggregate", std::make_shared()}, + }; } - -BlockInputStreams StorageSystemFunctions::read( - const Names & column_names, - const SelectQueryInfo &, - const Context &, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemFunctions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - const auto & functions = FunctionFactory::instance().functions; for (const auto & it : functions) { @@ -49,8 +36,6 @@ BlockInputStreams StorageSystemFunctions::read( res_columns[0]->insert(it.first); res_columns[1]->insert(UInt64(1)); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } } diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index f77b9536453..9735f756ee4 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -1,7 +1,9 @@ #pragma once +#include +#include #include -#include +#include namespace DB @@ -13,25 +15,17 @@ class Context; /** Implements `functions`system table, which allows you to get a list * all normal and aggregate functions. */ -class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorage + class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemFunctions"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemFunctions(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; -private: - const std::string name; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index c9ea685366b..7eab731bd12 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -124,10 +124,9 @@ static Strings getAllGraphiteSections(const AbstractConfiguration & config) } // namespace -StorageSystemGraphite::StorageSystemGraphite(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemGraphite::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { {"config_name", std::make_shared()}, {"regexp", std::make_shared()}, {"function", std::make_shared()}, @@ -135,23 +134,12 @@ StorageSystemGraphite::StorageSystemGraphite(const std::string & name_) {"precision", std::make_shared()}, {"priority", std::make_shared()}, {"is_default", std::make_shared()}, - })); + }; } -BlockInputStreams StorageSystemGraphite::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t /*max_block_size*/, - unsigned /*num_streams*/) +void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - const auto & config = context.getConfigRef(); Strings sections = getAllGraphiteSections(config); @@ -172,8 +160,6 @@ BlockInputStreams StorageSystemGraphite::read( } } } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.h b/dbms/src/Storages/System/StorageSystemGraphite.h index 8c7a625de54..7aa853ae9a7 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.h +++ b/dbms/src/Storages/System/StorageSystemGraphite.h @@ -1,31 +1,25 @@ #pragma once -#include +#include +#include +#include #include namespace DB { /// Provides information about Graphite configuration. -class StorageSystemGraphite : public ext::shared_ptr_helper, public IStorage +class StorageSystemGraphite : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemGraphite"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemGraphite(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemMacros.cpp b/dbms/src/Storages/System/StorageSystemMacros.cpp index 456730bde4b..ef19cc6db2e 100644 --- a/dbms/src/Storages/System/StorageSystemMacros.cpp +++ b/dbms/src/Storages/System/StorageSystemMacros.cpp @@ -1,38 +1,21 @@ -#include -#include -#include -#include -#include #include +#include +#include namespace DB { - -StorageSystemMacros::StorageSystemMacros(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemMacros::getNamesAndTypes() { - setColumns(ColumnsDescription({ - {"macro", std::make_shared()}, - {"substitution", std::make_shared()}, - })); + return { + {"macro", std::make_shared()}, + {"substitution", std::make_shared()}, + }; } - -BlockInputStreams StorageSystemMacros::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemMacros::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - auto macros = context.getMacros(); for (const auto & macro : macros->getMacroMap()) @@ -40,9 +23,6 @@ BlockInputStreams StorageSystemMacros::read( res_columns[0]->insert(macro.first); res_columns[1]->insert(macro.second); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemMacros.h b/dbms/src/Storages/System/StorageSystemMacros.h index d4bb5ab3732..fdc091dfe1b 100644 --- a/dbms/src/Storages/System/StorageSystemMacros.h +++ b/dbms/src/Storages/System/StorageSystemMacros.h @@ -1,7 +1,8 @@ #pragma once +#include #include -#include +#include namespace DB @@ -12,25 +13,17 @@ class Context; /** Information about macros for introspection. */ -class StorageSystemMacros : public ext::shared_ptr_helper, public IStorage +class StorageSystemMacros : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMacros"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemMacros(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemMerges.cpp b/dbms/src/Storages/System/StorageSystemMerges.cpp index d3af993e29e..29d54b83e32 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.cpp +++ b/dbms/src/Storages/System/StorageSystemMerges.cpp @@ -1,53 +1,36 @@ -#include -#include -#include -#include -#include -#include #include #include +#include namespace DB { -StorageSystemMerges::StorageSystemMerges(const std::string & name) - : name{name} +NamesAndTypesList StorageSystemMerges::getNamesAndTypes() { - setColumns(ColumnsDescription({ - { "database", std::make_shared() }, - { "table", std::make_shared() }, - { "elapsed", std::make_shared() }, - { "progress", std::make_shared() }, - { "num_parts", std::make_shared() }, - { "source_part_names", std::make_shared(std::make_shared()) }, - { "result_part_name", std::make_shared() }, - { "total_size_bytes_compressed", std::make_shared() }, - { "total_size_marks", std::make_shared() }, - { "bytes_read_uncompressed", std::make_shared() }, - { "rows_read", std::make_shared() }, - { "bytes_written_uncompressed", std::make_shared() }, - { "rows_written", std::make_shared() }, - { "columns_written", std::make_shared() }, - { "memory_usage", std::make_shared() }, - { "thread_number", std::make_shared() }, - })); + return { + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"elapsed", std::make_shared()}, + {"progress", std::make_shared()}, + {"num_parts", std::make_shared()}, + {"source_part_names", std::make_shared(std::make_shared())}, + {"result_part_name", std::make_shared()}, + {"total_size_bytes_compressed", std::make_shared()}, + {"total_size_marks", std::make_shared()}, + {"bytes_read_uncompressed", std::make_shared()}, + {"rows_read", std::make_shared()}, + {"bytes_written_uncompressed", std::make_shared()}, + {"rows_written", std::make_shared()}, + {"columns_written", std::make_shared()}, + {"memory_usage", std::make_shared()}, + {"thread_number", std::make_shared()}, + }; } -BlockInputStreams StorageSystemMerges::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t, - const unsigned) +void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (const auto & merge : context.getMergeList().get()) { size_t i = 0; @@ -68,8 +51,6 @@ BlockInputStreams StorageSystemMerges::read( res_columns[i++]->insert(merge.memory_usage); res_columns[i++]->insert(merge.thread_number); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } } diff --git a/dbms/src/Storages/System/StorageSystemMerges.h b/dbms/src/Storages/System/StorageSystemMerges.h index d48c97bfa17..f45f895d661 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.h +++ b/dbms/src/Storages/System/StorageSystemMerges.h @@ -1,7 +1,10 @@ #pragma once +#include +#include +#include #include -#include +#include namespace DB @@ -10,25 +13,17 @@ namespace DB class Context; -class StorageSystemMerges : public ext::shared_ptr_helper, public IStorage +class StorageSystemMerges : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMerges"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemMerges(const std::string & name); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemMetrics.cpp b/dbms/src/Storages/System/StorageSystemMetrics.cpp index 9d3b1cc9fbc..acfbd1b7340 100644 --- a/dbms/src/Storages/System/StorageSystemMetrics.cpp +++ b/dbms/src/Storages/System/StorageSystemMetrics.cpp @@ -1,39 +1,23 @@ + #include -#include -#include #include #include -#include #include namespace DB { - -StorageSystemMetrics::StorageSystemMetrics(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemMetrics::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { {"metric", std::make_shared()}, - {"value", std::make_shared()}, - })); + {"value", std::make_shared()}, + }; } - -BlockInputStreams StorageSystemMetrics::read( - const Names & column_names, - const SelectQueryInfo &, - const Context &, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemMetrics::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) { Int64 value = CurrentMetrics::values[i].load(std::memory_order_relaxed); @@ -41,9 +25,6 @@ BlockInputStreams StorageSystemMetrics::read( res_columns[0]->insert(String(CurrentMetrics::getDescription(CurrentMetrics::Metric(i)))); res_columns[1]->insert(value); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemMetrics.h b/dbms/src/Storages/System/StorageSystemMetrics.h index 7b6058de9e5..f74db926126 100644 --- a/dbms/src/Storages/System/StorageSystemMetrics.h +++ b/dbms/src/Storages/System/StorageSystemMetrics.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -12,25 +12,17 @@ class Context; /** Implements `metrics` system table, which provides information about the operation of the server. */ -class StorageSystemMetrics : public ext::shared_ptr_helper, public IStorage +class StorageSystemMetrics : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemMetrics"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemMetrics(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemModels.cpp b/dbms/src/Storages/System/StorageSystemModels.cpp index 5175989b861..2479742c8ec 100644 --- a/dbms/src/Storages/System/StorageSystemModels.cpp +++ b/dbms/src/Storages/System/StorageSystemModels.cpp @@ -2,45 +2,29 @@ #include #include #include -#include -#include -#include #include #include #include namespace DB { -StorageSystemModels::StorageSystemModels(const std::string & name) - : name{name} +NamesAndTypesList StorageSystemModels::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { { "name", std::make_shared() }, { "origin", std::make_shared() }, { "type", std::make_shared() }, { "creation_time", std::make_shared() }, { "last_exception", std::make_shared() }, - })); + }; } - -BlockInputStreams StorageSystemModels::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t, - const unsigned) +void StorageSystemModels::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - const auto & external_models = context.getExternalModels(); auto objects_map = external_models.getObjectsMap(); const auto & models = objects_map.get(); - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (const auto & model_info : models) { res_columns[0]->insert(model_info.first); @@ -73,8 +57,6 @@ BlockInputStreams StorageSystemModels::read( else res_columns[4]->insertDefault(); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } } diff --git a/dbms/src/Storages/System/StorageSystemModels.h b/dbms/src/Storages/System/StorageSystemModels.h index b32c5a804ce..ef30bd511ea 100644 --- a/dbms/src/Storages/System/StorageSystemModels.h +++ b/dbms/src/Storages/System/StorageSystemModels.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -10,25 +10,17 @@ namespace DB class Context; -class StorageSystemModels : public ext::shared_ptr_helper, public IStorage +class StorageSystemModels : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemModels"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemModels(const std::string & name); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemMutations.cpp b/dbms/src/Storages/System/StorageSystemMutations.cpp index 3cf204c6a77..17580c00940 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.cpp +++ b/dbms/src/Storages/System/StorageSystemMutations.cpp @@ -13,10 +13,10 @@ namespace DB { -StorageSystemMutations::StorageSystemMutations(const std::string & name_) - : name(name_) + +NamesAndTypesList StorageSystemMutations::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { { "database", std::make_shared() }, { "table", std::make_shared() }, { "mutation_id", std::make_shared() }, @@ -28,21 +28,12 @@ StorageSystemMutations::StorageSystemMutations(const std::string & name_) std::make_shared()) }, { "parts_to_do", std::make_shared() }, { "is_done", std::make_shared() }, - })); + }; } -BlockInputStreams StorageSystemMutations::read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemMutations::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - /// Collect a set of *MergeTree tables. std::map> merge_tree_tables; for (const auto & db : context.getDatabases()) @@ -83,13 +74,12 @@ BlockInputStreams StorageSystemMutations::read( VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) - return BlockInputStreams(); + return; col_database = filtered_block.getByName("database").column; col_table = filtered_block.getByName("table").column; } - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); for (size_t i_storage = 0; i_storage < col_database->size(); ++i_storage) { auto database = (*col_database)[i_storage].safeGet(); @@ -129,12 +119,6 @@ BlockInputStreams StorageSystemMutations::read( res_columns[col_num++]->insert(UInt64(status.is_done)); } } - - Block res = getSampleBlock().cloneEmpty(); - for (size_t i_col = 0; i_col < res.columns(); ++i_col) - res.getByPosition(i_col).column = std::move(res_columns[i_col]); - - return BlockInputStreams(1, std::make_shared(res)); } } diff --git a/dbms/src/Storages/System/StorageSystemMutations.h b/dbms/src/Storages/System/StorageSystemMutations.h index 3b82f3f46be..d2dcf99aa46 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.h +++ b/dbms/src/Storages/System/StorageSystemMutations.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -12,25 +12,17 @@ class Context; /// Implements the `mutations` system table, which provides information about the status of mutations /// in the MergeTree tables. -class StorageSystemMutations : public ext::shared_ptr_helper, public IStorage +class StorageSystemMutations : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: String getName() const override { return "SystemMutations"; } - String getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const String name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemMutations(const String & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.cpp b/dbms/src/Storages/System/StorageSystemProcesses.cpp index 793e3124a2a..f82c6554492 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.cpp +++ b/dbms/src/Storages/System/StorageSystemProcesses.cpp @@ -1,75 +1,60 @@ -#include #include #include -#include +#include #include #include -#include namespace DB { - -StorageSystemProcesses::StorageSystemProcesses(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemProcesses::getNamesAndTypes() { - setColumns(ColumnsDescription({ - { "is_initial_query", std::make_shared() }, + return { + {"is_initial_query", std::make_shared()}, - { "user", std::make_shared() }, - { "query_id", std::make_shared() }, - { "address", std::make_shared() }, - { "port", std::make_shared() }, + {"user", std::make_shared()}, + {"query_id", std::make_shared()}, + {"address", std::make_shared()}, + {"port", std::make_shared()}, - { "initial_user", std::make_shared() }, - { "initial_query_id", std::make_shared() }, - { "initial_address", std::make_shared() }, - { "initial_port", std::make_shared() }, + {"initial_user", std::make_shared()}, + {"initial_query_id", std::make_shared()}, + {"initial_address", std::make_shared()}, + {"initial_port", std::make_shared()}, - { "interface", std::make_shared() }, + {"interface", std::make_shared()}, - { "os_user", std::make_shared() }, - { "client_hostname", std::make_shared() }, - { "client_name", std::make_shared() }, - { "client_version_major", std::make_shared() }, - { "client_version_minor", std::make_shared() }, - { "client_revision", std::make_shared() }, + {"os_user", std::make_shared()}, + {"client_hostname", std::make_shared()}, + {"client_name", std::make_shared()}, + {"client_version_major", std::make_shared()}, + {"client_version_minor", std::make_shared()}, + {"client_revision", std::make_shared()}, - { "http_method", std::make_shared() }, - { "http_user_agent", std::make_shared() }, + {"http_method", std::make_shared()}, + {"http_user_agent", std::make_shared()}, - { "quota_key", std::make_shared() }, + {"quota_key", std::make_shared()}, - { "elapsed", std::make_shared() }, - { "is_cancelled", std::make_shared() }, - { "read_rows", std::make_shared() }, - { "read_bytes", std::make_shared() }, - { "total_rows_approx", std::make_shared() }, - { "written_rows", std::make_shared() }, - { "written_bytes", std::make_shared() }, - { "memory_usage", std::make_shared() }, - { "peak_memory_usage", std::make_shared() }, - { "query", std::make_shared() }, - })); + {"elapsed", std::make_shared()}, + {"is_cancelled", std::make_shared()}, + {"read_rows", std::make_shared()}, + {"read_bytes", std::make_shared()}, + {"total_rows_approx", std::make_shared()}, + {"written_rows", std::make_shared()}, + {"written_bytes", std::make_shared()}, + {"memory_usage", std::make_shared()}, + {"peak_memory_usage", std::make_shared()}, + {"query", std::make_shared()}, + }; } -BlockInputStreams StorageSystemProcesses::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemProcesses::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - ProcessList::Info info = context.getProcessList().getInfo(); - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (const auto & process : info) { size_t i = 0; @@ -103,9 +88,6 @@ BlockInputStreams StorageSystemProcesses::read( res_columns[i++]->insert(process.peak_memory_usage); res_columns[i++]->insert(process.query); } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index f8f26d13d35..3cbe0028af3 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -12,25 +12,17 @@ class Context; /** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ -class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorage +class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemProcesses"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemProcesses(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index 69fc73bd89c..51b0805c4c2 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -17,10 +16,10 @@ namespace DB { -StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string & name_) - : name(name_) + +NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes() { - setColumns(ColumnsDescription({ + return { /// Table properties. { "database", std::make_shared() }, { "table", std::make_shared() }, @@ -43,21 +42,12 @@ StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string & { "num_postponed", std::make_shared() }, { "postpone_reason", std::make_shared() }, { "last_postpone_time", std::make_shared() }, - })); + }; } -BlockInputStreams StorageSystemReplicationQueue::read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - std::map> replicated_tables; for (const auto & db : context.getDatabases()) for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) @@ -90,7 +80,7 @@ BlockInputStreams StorageSystemReplicationQueue::read( VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) - return BlockInputStreams(); + return; col_database_to_filter = filtered_block.getByName("database").column; col_table_to_filter = filtered_block.getByName("table").column; @@ -99,8 +89,6 @@ BlockInputStreams StorageSystemReplicationQueue::read( StorageReplicatedMergeTree::LogEntriesData queue; String replica_name; - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (size_t i = 0, tables_size = col_database_to_filter->size(); i < tables_size; ++i) { String database = (*col_database_to_filter)[i].safeGet(); @@ -139,9 +127,6 @@ BlockInputStreams StorageSystemReplicationQueue::read( res_columns[col_num++]->insert(UInt64(entry.last_postpone_time)); } } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.h b/dbms/src/Storages/System/StorageSystemReplicationQueue.h index 8554361e0df..63dc58118cd 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.h +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -12,25 +12,17 @@ class Context; /** Implements the `replication_queue` system table, which allows you to view the replication queues for the replicated tables. */ -class StorageSystemReplicationQueue : public ext::shared_ptr_helper, public IStorage +class StorageSystemReplicationQueue : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemReplicationQueue"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemReplicationQueue(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemSettings.cpp b/dbms/src/Storages/System/StorageSystemSettings.cpp index efb50c559cc..fee9467f6f9 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.cpp +++ b/dbms/src/Storages/System/StorageSystemSettings.cpp @@ -1,8 +1,5 @@ -#include -#include #include #include -#include #include #include @@ -10,44 +7,27 @@ namespace DB { - -StorageSystemSettings::StorageSystemSettings(const std::string & name_) - : name(name_) +NamesAndTypesList StorageSystemSettings::getNamesAndTypes() { - setColumns(ColumnsDescription({ - { "name", std::make_shared() }, - { "value", std::make_shared() }, - { "changed", std::make_shared() }, - { "description", std::make_shared() }, - })); + return { + {"name", std::make_shared()}, + {"value", std::make_shared()}, + {"changed", std::make_shared()}, + {"description", std::make_shared()}, + }; } - -BlockInputStreams StorageSystemSettings::read( - const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - const Settings & settings = context.getSettingsRef(); - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - -#define ADD_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \ - res_columns[0]->insert(String(#NAME)); \ - res_columns[1]->insert(settings.NAME.toString()); \ +#define ADD_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \ + res_columns[0]->insert(String(#NAME)); \ + res_columns[1]->insert(settings.NAME.toString()); \ res_columns[2]->insert(UInt64(settings.NAME.changed)); \ res_columns[3]->insert(String(DESCRIPTION)); APPLY_FOR_SETTINGS(ADD_SETTING) #undef ADD_SETTING - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } - } diff --git a/dbms/src/Storages/System/StorageSystemSettings.h b/dbms/src/Storages/System/StorageSystemSettings.h index 153b9213ef8..e44e0abbcd4 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.h +++ b/dbms/src/Storages/System/StorageSystemSettings.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -12,25 +12,17 @@ class Context; /** implements system table "settings", which allows to get information about the current settings. */ -class StorageSystemSettings : public ext::shared_ptr_helper, public IStorage +class StorageSystemSettings : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemSettings"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemSettings(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } diff --git a/dbms/src/Storages/System/StorageSystemTableFunctions.cpp b/dbms/src/Storages/System/StorageSystemTableFunctions.cpp index aaf448a6559..15067bbc41f 100644 --- a/dbms/src/Storages/System/StorageSystemTableFunctions.cpp +++ b/dbms/src/Storages/System/StorageSystemTableFunctions.cpp @@ -3,7 +3,13 @@ #include namespace DB { -void StorageSystemTableFunctions::fillData(MutableColumns & res_columns) const + +NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes() +{ + return {{"name", std::make_shared()}}; +} + +void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { const auto & functions = TableFunctionFactory::instance().getAllTableFunctions(); for (const auto & pair : functions) @@ -11,4 +17,5 @@ void StorageSystemTableFunctions::fillData(MutableColumns & res_columns) const res_columns[0]->insert(pair.first); } } + } diff --git a/dbms/src/Storages/System/StorageSystemTableFunctions.h b/dbms/src/Storages/System/StorageSystemTableFunctions.h index c791ed216a8..413af0f5c66 100644 --- a/dbms/src/Storages/System/StorageSystemTableFunctions.h +++ b/dbms/src/Storages/System/StorageSystemTableFunctions.h @@ -5,23 +5,25 @@ #include namespace DB { + class StorageSystemTableFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { protected: - void fillData(MutableColumns & res_columns) const override; + + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; public: - using IStorageSystemOneBlock::IStorageSystemOneBlock; std::string getName() const override { return "SystemTableFunctions"; } - static NamesAndTypesList getNamesAndTypes() - { - return {{"name", std::make_shared()}}; - } + static NamesAndTypesList getNamesAndTypes(); + }; + } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp index e506802ec74..ba3a821667f 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp @@ -1,9 +1,6 @@ -#include -#include #include #include #include -#include #include #include #include @@ -19,10 +16,8 @@ namespace DB { -StorageSystemZooKeeper::StorageSystemZooKeeper(const std::string & name_) - : name(name_) -{ - setColumns(ColumnsDescription({ +NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() { + return { { "name", std::make_shared() }, { "value", std::make_shared() }, { "czxid", std::make_shared() }, @@ -37,7 +32,7 @@ StorageSystemZooKeeper::StorageSystemZooKeeper(const std::string & name_) { "numChildren", std::make_shared() }, { "pzxid", std::make_shared() }, { "path", std::make_shared() }, - })); + }; } @@ -103,17 +98,8 @@ static String extractPath(const ASTPtr & query) } -BlockInputStreams StorageSystemZooKeeper::read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) +void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - check(column_names); - processed_stage = QueryProcessingStage::FetchColumns; - String path = extractPath(query_info.query); if (path.empty()) throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause."); @@ -136,8 +122,6 @@ BlockInputStreams StorageSystemZooKeeper::read( for (const String & node : nodes) futures.push_back(zookeeper->asyncTryGet(path_part + '/' + node)); - MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); - for (size_t i = 0, size = nodes.size(); i < size; ++i) { auto res = futures[i].get(); @@ -162,8 +146,6 @@ BlockInputStreams StorageSystemZooKeeper::read( res_columns[col_num++]->insert(Int64(stat.pzxid)); res_columns[col_num++]->insert(path); /// This is the original path. In order to process the request, condition in WHERE should be triggered. } - - return BlockInputStreams(1, std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.h b/dbms/src/Storages/System/StorageSystemZooKeeper.h index 45625ebab12..9644fe96162 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.h +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -12,25 +12,17 @@ class Context; /** Implements `zookeeper` system table, which allows you to view the data in ZooKeeper for debugging purposes. */ -class StorageSystemZooKeeper : public ext::shared_ptr_helper, public IStorage +class StorageSystemZooKeeper : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemZooKeeper"; } - std::string getTableName() const override { return name; } - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - -private: - const std::string name; + static NamesAndTypesList getNamesAndTypes(); protected: - StorageSystemZooKeeper(const std::string & name_); + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; }; } From 4905c6ac51f1b0099d8776a5b3685a9b4de4b13a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Jul 2018 17:32:54 +0300 Subject: [PATCH 09/17] CLICKHOUSE-3772: Move headers to appropriate place --- dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp | 2 ++ dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h | 3 --- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index f6c8e1c9953..059ef708a81 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h index e0e45e25a50..853cb97c974 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -1,8 +1,5 @@ #pragma once -#include -#include - #include #include From 8536761a76185f1242993dca2a738f8b69c4dd4f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 24 Jul 2018 17:42:52 +0300 Subject: [PATCH 10/17] CLICKHOUSE-3772: Move headers to appropriate place (one more time) --- dbms/src/Storages/System/StorageSystemBuildOptions.cpp | 3 ++- dbms/src/Storages/System/StorageSystemBuildOptions.h | 1 - dbms/src/Storages/System/StorageSystemClusters.cpp | 4 +++- .../Storages/System/StorageSystemDataTypeFamilies.cpp | 6 ++++-- dbms/src/Storages/System/StorageSystemDataTypeFamilies.h | 6 ++---- dbms/src/Storages/System/StorageSystemDatabases.cpp | 1 + dbms/src/Storages/System/StorageSystemDatabases.h | 3 +-- dbms/src/Storages/System/StorageSystemDictionaries.cpp | 9 +++++++-- dbms/src/Storages/System/StorageSystemDictionaries.h | 4 ---- dbms/src/Storages/System/StorageSystemEvents.cpp | 2 ++ dbms/src/Storages/System/StorageSystemEvents.h | 2 -- dbms/src/Storages/System/StorageSystemFormats.cpp | 2 ++ dbms/src/Storages/System/StorageSystemFormats.h | 2 -- dbms/src/Storages/System/StorageSystemFunctions.cpp | 3 --- dbms/src/Storages/System/StorageSystemFunctions.h | 2 -- dbms/src/Storages/System/StorageSystemGraphite.h | 1 - dbms/src/Storages/System/StorageSystemMacros.cpp | 2 +- 17 files changed, 25 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.cpp b/dbms/src/Storages/System/StorageSystemBuildOptions.cpp index 621854f1a5f..2a8ffc947be 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.cpp +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.cpp @@ -1,6 +1,7 @@ +#include +#include #include #include -#include namespace DB { diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.h b/dbms/src/Storages/System/StorageSystemBuildOptions.h index a8909addf85..749ffbddbaf 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.h +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemClusters.cpp b/dbms/src/Storages/System/StorageSystemClusters.cpp index 693d127b7c7..a44c0ff4c2d 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.cpp +++ b/dbms/src/Storages/System/StorageSystemClusters.cpp @@ -1,7 +1,9 @@ +#include +#include +#include #include #include #include -#include namespace DB { diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index 7f154e9f17b..126f98e506f 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include namespace DB @@ -9,8 +11,8 @@ NamesAndTypesList StorageSystemDataTypeFamilies::getNamesAndTypes() { return { {"name", std::make_shared()}, - {"case_insensivie", std::make_shared(std::make_shared())}, - {"alias_to", std::make_shared(std::make_shared())}, + {"case_insensivie", std::make_shared()}, + {"alias_to", std::make_shared()}, }; } diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h index 3cc1c50d426..365e2790699 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -1,10 +1,8 @@ #pragma once -#include -#include -#include -#include #include +#include + namespace DB { diff --git a/dbms/src/Storages/System/StorageSystemDatabases.cpp b/dbms/src/Storages/System/StorageSystemDatabases.cpp index 7145cb36178..4df3d360a3b 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.cpp +++ b/dbms/src/Storages/System/StorageSystemDatabases.cpp @@ -1,4 +1,5 @@ #include +#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemDatabases.h b/dbms/src/Storages/System/StorageSystemDatabases.h index 26052c726c5..c83f5a72efc 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.h +++ b/dbms/src/Storages/System/StorageSystemDatabases.h @@ -1,8 +1,7 @@ #pragma once -#include -#include #include +#include namespace DB diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index ae52af89689..3a8be583599 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -1,9 +1,14 @@ -#include -#include +#include +#include +#include +#include #include #include #include +#include #include +#include + #include #include diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.h b/dbms/src/Storages/System/StorageSystemDictionaries.h index 57512692809..87df9ceada7 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.h +++ b/dbms/src/Storages/System/StorageSystemDictionaries.h @@ -1,9 +1,5 @@ #pragma once -#include -#include -#include -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemEvents.cpp b/dbms/src/Storages/System/StorageSystemEvents.cpp index 02b7c776086..eb4832c0c92 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.cpp +++ b/dbms/src/Storages/System/StorageSystemEvents.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include namespace DB diff --git a/dbms/src/Storages/System/StorageSystemEvents.h b/dbms/src/Storages/System/StorageSystemEvents.h index 3c3469c3656..5b02b7739f1 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.h +++ b/dbms/src/Storages/System/StorageSystemEvents.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemFormats.cpp b/dbms/src/Storages/System/StorageSystemFormats.cpp index 13df91b6fc4..96ce7ea7ed9 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.cpp +++ b/dbms/src/Storages/System/StorageSystemFormats.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemFormats.h b/dbms/src/Storages/System/StorageSystemFormats.h index 9b040ada1ca..82f8303b5b0 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.h +++ b/dbms/src/Storages/System/StorageSystemFormats.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemFunctions.cpp b/dbms/src/Storages/System/StorageSystemFunctions.cpp index 0c22a022cb0..4737f99aaf4 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.cpp +++ b/dbms/src/Storages/System/StorageSystemFunctions.cpp @@ -1,7 +1,4 @@ #include -#include -#include -#include #include #include #include diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index 9735f756ee4..a40992c1bc9 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemGraphite.h b/dbms/src/Storages/System/StorageSystemGraphite.h index 7aa853ae9a7..fa63c839857 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.h +++ b/dbms/src/Storages/System/StorageSystemGraphite.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemMacros.cpp b/dbms/src/Storages/System/StorageSystemMacros.cpp index ef19cc6db2e..8e6420add8b 100644 --- a/dbms/src/Storages/System/StorageSystemMacros.cpp +++ b/dbms/src/Storages/System/StorageSystemMacros.cpp @@ -1,6 +1,6 @@ +#include #include #include -#include namespace DB From 1824758f8f920bdc24a89d4d260b01a3026c0c3e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 24 Jul 2018 21:37:07 +0300 Subject: [PATCH 11/17] Update StorageSystemAggregateFunctionCombinators.cpp --- .../System/StorageSystemAggregateFunctionCombinators.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp index 3fb91c6ce8a..8fa335faceb 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.cpp @@ -18,10 +18,7 @@ void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_co for (const auto & pair : combinators) { res_columns[0]->insert(pair.first); - if (pair.second->isForInternalUsageOnly()) - res_columns[1]->insert(UInt64(1)); - else - res_columns[1]->insert(UInt64(0)); + res_columns[1]->insert(UInt64(pair.second->isForInternalUsageOnly())); } } From 2834ba2dfbab7e63b6f527bec2d7b6505fadf4f0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 24 Jul 2018 21:38:41 +0300 Subject: [PATCH 12/17] Update StorageSystemDictionaries.cpp --- dbms/src/Storages/System/StorageSystemDictionaries.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index 3a8be583599..665b992c829 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -35,7 +35,8 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() }; } -void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { +void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +{ const auto & external_dictionaries = context.getExternalDictionaries(); auto objects_map = external_dictionaries.getObjectsMap(); const auto & dictionaries = objects_map.get(); From 8af95f39e8d9d6a7a2c5ed7a71504fda9eeff08c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 24 Jul 2018 21:39:21 +0300 Subject: [PATCH 13/17] Update StorageSystemFunctions.h --- dbms/src/Storages/System/StorageSystemFunctions.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index a40992c1bc9..baead3d8186 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -13,7 +13,7 @@ class Context; /** Implements `functions`system table, which allows you to get a list * all normal and aggregate functions. */ - class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { public: std::string getName() const override { return "SystemFunctions"; } From ba4f988e3f8bfb76af09c22ed6fb7285136ec19f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 24 Jul 2018 21:40:53 +0300 Subject: [PATCH 14/17] Update StorageSystemZooKeeper.cpp --- dbms/src/Storages/System/StorageSystemZooKeeper.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp index ba3a821667f..612b0d782d7 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp @@ -16,7 +16,8 @@ namespace DB { -NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() { +NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() +{ return { { "name", std::make_shared() }, { "value", std::make_shared() }, From 9adb2516897cc6a2b800775b0fbc9f0c41280397 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Jul 2018 19:08:23 +0300 Subject: [PATCH 15/17] CLICKHOUSE-3857: Add table table_engines, alias factory base class and columns alias_to and case_insensitive to functions --- .../AggregateFunctionBitwise.cpp | 6 +- .../AggregateFunctionFactory.cpp | 8 +- .../AggregateFunctionFactory.h | 31 +++-- .../AggregateFunctionQuantile.cpp | 32 ++--- .../AggregateFunctionsStatisticsSimple.cpp | 12 +- dbms/src/Common/IFactoryWithAliases.h | 125 ++++++++++++++++++ dbms/src/DataTypes/DataTypeFactory.cpp | 65 +-------- dbms/src/DataTypes/DataTypeFactory.h | 38 ++---- dbms/src/Functions/FunctionFactory.cpp | 12 +- dbms/src/Functions/FunctionFactory.h | 19 ++- dbms/src/Functions/FunctionsRound.cpp | 4 +- dbms/src/Storages/StorageFactory.h | 5 + .../System/StorageSystemDataTypeFamilies.cpp | 2 +- .../System/StorageSystemFunctions.cpp | 33 +++-- .../System/StorageSystemTableEngines.cpp | 22 +++ .../System/StorageSystemTableEngines.h | 27 ++++ .../Storages/System/attachSystemTables.cpp | 2 + 17 files changed, 275 insertions(+), 168 deletions(-) create mode 100644 dbms/src/Common/IFactoryWithAliases.h create mode 100644 dbms/src/Storages/System/StorageSystemTableEngines.cpp create mode 100644 dbms/src/Storages/System/StorageSystemTableEngines.h diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp index 762baf2451b..8c188bcbb8e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp @@ -38,9 +38,9 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory) factory.registerFunction("groupBitXor", createAggregateFunctionBitwise); /// Aliases for compatibility with MySQL. - factory.registerFunction("BIT_OR", createAggregateFunctionBitwise, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("BIT_AND", createAggregateFunctionBitwise, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("BIT_XOR", createAggregateFunctionBitwise, AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 366aef1f829..353b5a213b3 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -78,11 +78,12 @@ AggregateFunctionPtr AggregateFunctionFactory::get( AggregateFunctionPtr AggregateFunctionFactory::getImpl( - const String & name, + const String & name_param, const DataTypes & argument_types, const Array & parameters, int recursion_level) const { + String name = getAliasToOrName(name_param); /// Find by exact match. auto it = aggregate_functions.find(name); if (it != aggregate_functions.end()) @@ -126,10 +127,11 @@ AggregateFunctionPtr AggregateFunctionFactory::tryGet(const String & name, const bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int recursion_level) const { - if (aggregate_functions.count(name)) + if (aggregate_functions.count(name) || isAlias(name)) return true; - if (recursion_level == 0 && case_insensitive_aggregate_functions.count(Poco::toLower(name))) + String name_lowercase = Poco::toLower(name); + if (recursion_level == 0 && (case_insensitive_aggregate_functions.count(name_lowercase) || isAlias(name_lowercase))) return true; if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name)) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.h b/dbms/src/AggregateFunctions/AggregateFunctionFactory.h index bc36e76c11f..92598e52509 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -20,27 +21,18 @@ class IDataType; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; +/** Creator have arguments: name of aggregate function, types of arguments, values of parameters. + * Parameters are for "parametric" aggregate functions. + * For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments". + */ +using AggregateFunctionCreator = std::function; + /** Creates an aggregate function by name. */ -class AggregateFunctionFactory final : public ext::singleton +class AggregateFunctionFactory final : public ext::singleton, public IFactoryWithAliases { - friend class StorageSystemFunctions; - public: - /** Creator have arguments: name of aggregate function, types of arguments, values of parameters. - * Parameters are for "parametric" aggregate functions. - * For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments". - */ - using Creator = std::function; - - /// For compatibility with SQL, it's possible to specify that certain aggregate function name is case insensitive. - enum CaseSensitiveness - { - CaseSensitive, - CaseInsensitive - }; - /// Register a function by its name. /// No locking, you must register all functions before usage of get. void registerFunction( @@ -77,6 +69,13 @@ private: /// Case insensitive aggregate functions will be additionally added here with lowercased name. AggregateFunctions case_insensitive_aggregate_functions; + + const AggregateFunctions & getCreatorMap() const override { return aggregate_functions; } + + const AggregateFunctions & getCaseInsensitiveCreatorMap() const override { return case_insensitive_aggregate_functions; } + + String getFactoryName() const override { return "AggregateFunctionFactory"; } + }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 250ee422e8b..62455af6353 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -93,30 +93,14 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory) createAggregateFunctionQuantile); /// 'median' is an alias for 'quantile' - - factory.registerFunction("median", - createAggregateFunctionQuantile); - - factory.registerFunction("medianDeterministic", - createAggregateFunctionQuantile); - - factory.registerFunction("medianExact", - createAggregateFunctionQuantile); - - factory.registerFunction("medianExactWeighted", - createAggregateFunctionQuantile); - - factory.registerFunction("medianTiming", - createAggregateFunctionQuantile); - - factory.registerFunction("medianTimingWeighted", - createAggregateFunctionQuantile); - - factory.registerFunction("medianTDigest", - createAggregateFunctionQuantile); - - factory.registerFunction("medianTDigestWeighted", - createAggregateFunctionQuantile); + factory.registerAlias("median", NameQuantile::name); + factory.registerAlias("medianDeterministic", NameQuantileDeterministic::name); + factory.registerAlias("medianExact", NameQuantileExact::name); + factory.registerAlias("medianExactWeighted", NameQuantileExactWeighted::name); + factory.registerAlias("medianTiming", NameQuantileTiming::name); + factory.registerAlias("medianTimingWeighted", NameQuantileTimingWeighted::name); + factory.registerAlias("medianTDigest", NameQuantileTDigest::name); + factory.registerAlias("medianTDigestWeighted", NameQuantileTDigestWeighted::name); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp b/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp index 089ea59cd79..c42372187bc 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp @@ -56,12 +56,12 @@ void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory & facto factory.registerFunction("corr", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::CaseInsensitive); /// Synonims for compatibility. - factory.registerFunction("VAR_SAMP", createAggregateFunctionStatisticsUnary, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("VAR_POP", createAggregateFunctionStatisticsUnary, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("STDDEV_SAMP", createAggregateFunctionStatisticsUnary, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("STDDEV_POP", createAggregateFunctionStatisticsUnary, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("COVAR_SAMP", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("COVAR_POP", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/Common/IFactoryWithAliases.h b/dbms/src/Common/IFactoryWithAliases.h new file mode 100644 index 00000000000..dc341782894 --- /dev/null +++ b/dbms/src/Common/IFactoryWithAliases.h @@ -0,0 +1,125 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/** If stored objects may have several names (aliases) + * this interface may be helpful + * template parameter is available as Creator + */ +template +class IFactoryWithAliases +{ +protected: + using Creator = CreatorFunc; + + String getAliasToOrName(const String & name) const + { + if (aliases.count(name)) + return aliases.at(name); + else if (String name_lowercase = Poco::toLower(name); case_insensitive_aliases.count(name_lowercase)) + return case_insensitive_aliases.at(name_lowercase); + else + return name; + } + +public: + /// For compatibility with SQL, it's possible to specify that certain function name is case insensitive. + enum CaseSensitiveness + { + CaseSensitive, + CaseInsensitive + }; + + /** Register additional name for creater + * real_name have to be already registered. + */ + void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive) + { + const auto & creator_map = getCreatorMap(); + const auto & case_insensitive_creator_map = getCaseInsensitiveCreatorMap(); + const String factory_name = getFactoryName(); + + String real_dict_name; + if (creator_map.count(real_name)) + real_dict_name = real_name; + else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.count(real_name_lowercase)) + real_dict_name = real_name_lowercase; + else + throw Exception(factory_name + ": can't create alias '" + alias_name + "', the real name '" + real_name + "' is not registered", + ErrorCodes::LOGICAL_ERROR); + + String alias_name_lowercase = Poco::toLower(alias_name); + + if (creator_map.count(alias_name) || case_insensitive_creator_map.count(alias_name_lowercase)) + throw Exception( + factory_name + ": the alias name '" + alias_name + "' is already registered as real name", ErrorCodes::LOGICAL_ERROR); + + if (case_sensitiveness == CaseInsensitive) + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second) + throw Exception( + factory_name + ": case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + + if (!aliases.emplace(alias_name, real_dict_name).second) + throw Exception(factory_name + ": alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + } + + std::vector getAllRegisteredNames() const + { + std::vector result; + auto getter = [](const auto & pair) { return pair.first; }; + std::transform(getCreatorMap().begin(), getCreatorMap().end(), std::back_inserter(result), getter); + std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter); + return result; + } + + bool isCaseInsensitive(const String & name) const + { + String name_lowercase = Poco::toLower(name); + return getCaseInsensitiveCreatorMap().count(name_lowercase) || case_insensitive_aliases.count(name_lowercase); + } + + const String & aliasTo(const String & name) const + { + if (auto it = aliases.find(name); it != aliases.end()) + return it->second; + else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end()) + return it->second; + + throw Exception(getFactoryName() + ": name '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR); + } + + bool isAlias(const String & name) const + { + return aliases.count(name) || case_insensitive_aliases.count(name); + } + + virtual ~IFactoryWithAliases() {} + +private: + using InnerMap = std::unordered_map; // name -> creator + using AliasMap = std::unordered_map; // alias -> original type + + virtual const InnerMap & getCreatorMap() const = 0; + virtual const InnerMap & getCaseInsensitiveCreatorMap() const = 0; + virtual String getFactoryName() const = 0; + + /// Alias map to data_types from previous two maps + AliasMap aliases; + + /// Case insensitive aliases + AliasMap case_insensitive_aliases; +}; + +} diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 87599a90370..9706ecf4944 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -51,8 +51,10 @@ DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const throw Exception("Unexpected AST element for data type.", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } -DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & parameters) const +DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr & parameters) const { + String family_name = getAliasToOrName(family_name_param); + { DataTypesDictionary::const_iterator it = data_types.find(family_name); if (data_types.end() != it) @@ -67,11 +69,6 @@ DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & para return it->second(parameters); } - if (auto it = aliases.find(family_name); it != aliases.end()) - return get(it->second, parameters); - else if (auto it = case_insensitive_aliases.find(family_name_lowercase); it != case_insensitive_aliases.end()) - return get(it->second, parameters); - throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE); } @@ -84,7 +81,7 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat String family_name_lowercase = Poco::toLower(family_name); - if (aliases.count(family_name) || case_insensitive_aliases.count(family_name_lowercase)) + if (isAlias(family_name) || isAlias(family_name_lowercase)) throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is already registered as alias", ErrorCodes::LOGICAL_ERROR); @@ -99,29 +96,6 @@ void DataTypeFactory::registerDataType(const String & family_name, Creator creat ErrorCodes::LOGICAL_ERROR); } -void DataTypeFactory::registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) -{ - String real_type_dict_name; - if (data_types.count(real_name)) - real_type_dict_name = real_name; - else if (auto type_name_lowercase = Poco::toLower(real_name); case_insensitive_data_types.count(type_name_lowercase)) - real_type_dict_name = type_name_lowercase; - else - throw Exception("DataTypeFactory: can't create alias '" + alias_name + "' the data type family '" + real_name + "' is not registered", ErrorCodes::LOGICAL_ERROR); - - String alias_name_lowercase = Poco::toLower(alias_name); - - if (data_types.count(alias_name) || case_insensitive_data_types.count(alias_name_lowercase)) - throw Exception("DataTypeFactory: the alias name " + alias_name + " is already registered as datatype", ErrorCodes::LOGICAL_ERROR); - - if (case_sensitiveness == CaseInsensitive) - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_type_dict_name).second) - throw Exception("DataTypeFactory: case insensitive alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - - if (!aliases.emplace(alias_name, real_type_dict_name).second) - throw Exception("DataTypeFactory: alias name '" + alias_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); -} - void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness) { if (creator == nullptr) @@ -136,37 +110,6 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator }, case_sensitiveness); } -std::vector DataTypeFactory::getAllDataTypeNames() const -{ - std::vector result; - auto getter = [] (const auto & pair) { return pair.first; }; - std::transform(data_types.begin(), data_types.end(), std::back_inserter(result), getter); - std::transform(aliases.begin(), aliases.end(), std::back_inserter(result), getter); - return result; -} - -bool DataTypeFactory::isCaseInsensitive(const String & name) const -{ - String name_lowercase = Poco::toLower(name); - return case_insensitive_data_types.count(name_lowercase) || case_insensitive_aliases.count(name_lowercase); -} - -const String & DataTypeFactory::aliasTo(const String & name) const -{ - if (auto it = aliases.find(name); it != aliases.end()) - return it->second; - else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end()) - return it->second; - - throw Exception("DataTypeFactory: the data type '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR); -} - - -bool DataTypeFactory::isAlias(const String & name) const -{ - return aliases.count(name) || case_insensitive_aliases.count(name); -} - void registerDataTypeNumbers(DataTypeFactory & factory); void registerDataTypeDate(DataTypeFactory & factory); void registerDataTypeDateTime(DataTypeFactory & factory); diff --git a/dbms/src/DataTypes/DataTypeFactory.h b/dbms/src/DataTypes/DataTypeFactory.h index 423ff6acd1e..21d22cf932e 100644 --- a/dbms/src/DataTypes/DataTypeFactory.h +++ b/dbms/src/DataTypes/DataTypeFactory.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -19,58 +20,37 @@ using ASTPtr = std::shared_ptr; /** Creates a data type by name of data type family and parameters. */ -class DataTypeFactory final : public ext::singleton +class DataTypeFactory final : public ext::singleton, public IFactoryWithAliases> { private: - using Creator = std::function; using SimpleCreator = std::function; using DataTypesDictionary = std::unordered_map; - using AliasMap = std::unordered_map; // alias -> original type public: DataTypePtr get(const String & full_name) const; DataTypePtr get(const String & family_name, const ASTPtr & parameters) const; DataTypePtr get(const ASTPtr & ast) const; - /// For compatibility with SQL, it's possible to specify that certain data type name is case insensitive. - enum CaseSensitiveness - { - CaseSensitive, - CaseInsensitive - }; - /// Register a type family by its name. void registerDataType(const String & family_name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); /// Register a simple data type, that have no parameters. void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - /** Register additional name for datatype. - * real_name datatype have to be already registered. - */ - void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); - - std::vector getAllDataTypeNames() const; - - bool isCaseInsensitive(const String & name) const; - - const String & aliasTo(const String & name) const; - - bool isAlias(const String & name) const; - private: DataTypesDictionary data_types; /// Case insensitive data types will be additionally added here with lowercased name. DataTypesDictionary case_insensitive_data_types; - /// Alias map to data_types from previous two maps - AliasMap aliases; - - /// Case insensitive aliases - AliasMap case_insensitive_aliases; - DataTypeFactory(); + + const DataTypesDictionary & getCreatorMap() const override { return data_types; } + + const DataTypesDictionary & getCaseInsensitiveCreatorMap() const override { return case_insensitive_data_types; } + + String getFactoryName() const override { return "DataTypeFactory"; } + friend class ext::singleton; }; diff --git a/dbms/src/Functions/FunctionFactory.cpp b/dbms/src/Functions/FunctionFactory.cpp index 9bb2abbb013..0b2f042089d 100644 --- a/dbms/src/Functions/FunctionFactory.cpp +++ b/dbms/src/Functions/FunctionFactory.cpp @@ -6,7 +6,6 @@ #include - namespace DB { @@ -26,8 +25,13 @@ void FunctionFactory::registerFunction(const throw Exception("FunctionFactory: the function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + String function_name_lowercase = Poco::toLower(name); + if (isAlias(name) || isAlias(function_name_lowercase)) + throw Exception("FunctionFactory: the function name '" + name + "' is already registered as alias", + ErrorCodes::LOGICAL_ERROR); + if (case_sensitiveness == CaseInsensitive - && !case_insensitive_functions.emplace(Poco::toLower(name), creator).second) + && !case_insensitive_functions.emplace(function_name_lowercase, creator).second) throw Exception("FunctionFactory: the case insensitive function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } @@ -45,9 +49,11 @@ FunctionBuilderPtr FunctionFactory::get( FunctionBuilderPtr FunctionFactory::tryGet( - const std::string & name, + const std::string & name_param, const Context & context) const { + String name = getAliasToOrName(name_param); + auto it = functions.find(name); if (functions.end() != it) return it->second(context); diff --git a/dbms/src/Functions/FunctionFactory.h b/dbms/src/Functions/FunctionFactory.h index a061c3103fd..7fa0f81f475 100644 --- a/dbms/src/Functions/FunctionFactory.h +++ b/dbms/src/Functions/FunctionFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -20,19 +21,9 @@ class Context; * Function could use for initialization (take ownership of shared_ptr, for example) * some dictionaries from Context. */ -class FunctionFactory : public ext::singleton +class FunctionFactory : public ext::singleton, public IFactoryWithAliases> { - friend class StorageSystemFunctions; - public: - using Creator = std::function; - - /// For compatibility with SQL, it's possible to specify that certain function name is case insensitive. - enum CaseSensitiveness - { - CaseSensitive, - CaseInsensitive - }; template void registerFunction(CaseSensitiveness case_sensitiveness = CaseSensitive) @@ -67,6 +58,12 @@ private: return std::make_shared(Function::create(context)); } + const Functions & getCreatorMap() const override { return functions; } + + const Functions & getCaseInsensitiveCreatorMap() const override { return case_insensitive_functions; } + + String getFactoryName() const override { return "FunctionFactory"; } + /// Register a function by its name. /// No locking, you must register all functions before usage of get. void registerFunction( diff --git a/dbms/src/Functions/FunctionsRound.cpp b/dbms/src/Functions/FunctionsRound.cpp index 7bf7eb791ad..9cb9e1001ae 100644 --- a/dbms/src/Functions/FunctionsRound.cpp +++ b/dbms/src/Functions/FunctionsRound.cpp @@ -16,8 +16,8 @@ void registerFunctionsRound(FunctionFactory & factory) factory.registerFunction("trunc", FunctionFactory::CaseInsensitive); /// Compatibility aliases. - factory.registerFunction("ceiling", FunctionFactory::CaseInsensitive); - factory.registerFunction("truncate", FunctionFactory::CaseInsensitive); + factory.registerAlias("ceiling", "ceil", FunctionFactory::CaseInsensitive); + factory.registerAlias("truncate", "trunc", FunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index 2acb9fb7c00..4addfcd9794 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -53,6 +53,11 @@ public: /// No locking, you must register all engines before usage of get. void registerStorage(const std::string & name, Creator creator); + const auto & getAllStorages() const + { + return storages; + } + private: using Storages = std::unordered_map; Storages storages; diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index 126f98e506f..68228ca3965 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -19,7 +19,7 @@ NamesAndTypesList StorageSystemDataTypeFamilies::getNamesAndTypes() void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { const auto & factory = DataTypeFactory::instance(); - auto names = factory.getAllDataTypeNames(); + auto names = factory.getAllRegisteredNames(); for (const auto & name : names) { res_columns[0]->insert(name); diff --git a/dbms/src/Storages/System/StorageSystemFunctions.cpp b/dbms/src/Storages/System/StorageSystemFunctions.cpp index 4737f99aaf4..f8bcb379981 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.cpp +++ b/dbms/src/Storages/System/StorageSystemFunctions.cpp @@ -9,30 +9,45 @@ namespace DB { +namespace +{ + template + void fillRow(MutableColumns & res_columns, const String & name, UInt64 is_aggregate, const Factory & f) + { + res_columns[0]->insert(name); + res_columns[1]->insert(is_aggregate); + res_columns[2]->insert(UInt64(f.isCaseInsensitive(name))); + if (f.isAlias(name)) + res_columns[3]->insert(f.aliasTo(name)); + else + res_columns[3]->insert(String{}); + } +} NamesAndTypesList StorageSystemFunctions::getNamesAndTypes() { return { {"name", std::make_shared()}, {"is_aggregate", std::make_shared()}, + {"case_insensivie", std::make_shared()}, + {"alias_to", std::make_shared()}, }; } void StorageSystemFunctions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - const auto & functions = FunctionFactory::instance().functions; - for (const auto & it : functions) + const auto & functions_factory = FunctionFactory::instance(); + const auto & function_names = functions_factory.getAllRegisteredNames(); + for (const auto & name : function_names) { - res_columns[0]->insert(it.first); - res_columns[1]->insert(UInt64(0)); + fillRow(res_columns, name, UInt64(0), functions_factory); } - const auto & aggregate_functions = AggregateFunctionFactory::instance().aggregate_functions; - for (const auto & it : aggregate_functions) + const auto & aggregate_functions_factory = AggregateFunctionFactory::instance(); + const auto & aggregate_function_names = aggregate_functions_factory.getAllRegisteredNames(); + for (const auto & name : aggregate_function_names) { - res_columns[0]->insert(it.first); - res_columns[1]->insert(UInt64(1)); + fillRow(res_columns, name, UInt64(1), aggregate_functions_factory); } } - } diff --git a/dbms/src/Storages/System/StorageSystemTableEngines.cpp b/dbms/src/Storages/System/StorageSystemTableEngines.cpp new file mode 100644 index 00000000000..99b0a650d68 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemTableEngines.cpp @@ -0,0 +1,22 @@ +#include +#include +#include + +namespace DB +{ + +NamesAndTypesList StorageSystemTableEngines::getNamesAndTypes() +{ + return {{"name", std::make_shared()}}; +} + +void StorageSystemTableEngines::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +{ + const auto & storages = StorageFactory::instance().getAllStorages(); + for (const auto & [name, creator] : storages) + { + res_columns[0]->insert(name); + } +} + +} diff --git a/dbms/src/Storages/System/StorageSystemTableEngines.h b/dbms/src/Storages/System/StorageSystemTableEngines.h new file mode 100644 index 00000000000..f0f6b62d59d --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemTableEngines.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class StorageSystemTableEngines : public ext::shared_ptr_helper, + public IStorageSystemOneBlock +{ +protected: + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; + + using IStorageSystemOneBlock::IStorageSystemOneBlock; + +public: + std::string getName() const override + { + return "SystemTableEngines"; + } + + static NamesAndTypesList getNamesAndTypes(); +}; + +} diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index e29a26df4eb..479337d1b41 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -52,6 +53,7 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("aggregate_function_combinators", StorageSystemAggregateFunctionCombinators::create("aggregate_function_combinators")); system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families")); system_database.attachTable("collations", StorageSystemCollations::create("collations")); + system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); } void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) From 388807739802a5f893ba9b5af0ead3cf1b7ed8b1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Jul 2018 19:19:11 +0300 Subject: [PATCH 16/17] CLICKHOUSE-3857: Fix typo --- dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp | 2 +- dbms/src/Storages/System/StorageSystemFunctions.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp index 68228ca3965..c8d692fddd8 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp @@ -11,7 +11,7 @@ NamesAndTypesList StorageSystemDataTypeFamilies::getNamesAndTypes() { return { {"name", std::make_shared()}, - {"case_insensivie", std::make_shared()}, + {"case_insensitive", std::make_shared()}, {"alias_to", std::make_shared()}, }; } diff --git a/dbms/src/Storages/System/StorageSystemFunctions.cpp b/dbms/src/Storages/System/StorageSystemFunctions.cpp index f8bcb379981..f63d0b9b932 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.cpp +++ b/dbms/src/Storages/System/StorageSystemFunctions.cpp @@ -29,7 +29,7 @@ NamesAndTypesList StorageSystemFunctions::getNamesAndTypes() return { {"name", std::make_shared()}, {"is_aggregate", std::make_shared()}, - {"case_insensivie", std::make_shared()}, + {"case_insensitive", std::make_shared()}, {"alias_to", std::make_shared()}, }; } From dc2c22ffff86b88ecc665f9d0543c2cf7e7eb705 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 25 Jul 2018 21:46:16 +0300 Subject: [PATCH 17/17] Update IFactoryWithAliases.h --- dbms/src/Common/IFactoryWithAliases.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/IFactoryWithAliases.h b/dbms/src/Common/IFactoryWithAliases.h index dc341782894..9006a3c7cfd 100644 --- a/dbms/src/Common/IFactoryWithAliases.h +++ b/dbms/src/Common/IFactoryWithAliases.h @@ -42,7 +42,7 @@ public: CaseInsensitive }; - /** Register additional name for creater + /** Register additional name for creator * real_name have to be already registered. */ void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)