mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
CLICKHOUSE-3772: Add system tables formats, data_type_families, table_functions, aggregate_function_combinators
This commit is contained in:
parent
d0d61e4ea4
commit
ec156e9f2b
@ -22,6 +22,10 @@ public:
|
||||
/// Example: if the name is 'avgIf', it will return combinator -If.
|
||||
AggregateFunctionCombinatorPtr tryFindSuffix(const std::string & name) const;
|
||||
|
||||
std::unordered_map<std::string, AggregateFunctionCombinatorPtr> getAllAggregateFunctionCombinators() const {
|
||||
return dict;
|
||||
}
|
||||
|
||||
private:
|
||||
std::unordered_map<std::string, AggregateFunctionCombinatorPtr> dict;
|
||||
};
|
||||
|
@ -44,6 +44,10 @@ public:
|
||||
/// Register a simple data type, that have no parameters.
|
||||
void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
|
||||
|
||||
DataTypesDictionary getAllDataTypes() const {
|
||||
return data_types;
|
||||
}
|
||||
|
||||
private:
|
||||
DataTypesDictionary data_types;
|
||||
|
||||
|
@ -58,6 +58,10 @@ public:
|
||||
void registerInputFormat(const String & name, InputCreator input_creator);
|
||||
void registerOutputFormat(const String & name, OutputCreator output_creator);
|
||||
|
||||
const FormatsDictionary & getAllFormats() const {
|
||||
return dict;
|
||||
}
|
||||
|
||||
private:
|
||||
FormatsDictionary dict;
|
||||
|
||||
|
53
dbms/src/Storages/System/IStorageSystemWithStringColumns.h
Normal file
53
dbms/src/Storages/System/IStorageSystemWithStringColumns.h
Normal file
@ -0,0 +1,53 @@
|
||||
#pragma once
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
template <typename Self>
|
||||
class IStorageSystemWithStringColumns : public IStorage
|
||||
{
|
||||
protected:
|
||||
virtual void fillData(MutableColumns & res_columns) const = 0;
|
||||
|
||||
public:
|
||||
IStorageSystemWithStringColumns (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<DataTypeString>()});
|
||||
}
|
||||
setColumns(ColumnsDescription(name_list));
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
check(column_names);
|
||||
processed_stage = QueryProcessingStage::FetchColumns;
|
||||
|
||||
MutableColumns res_columns = getSampleBlock().cloneEmptyColumns();
|
||||
fillData(res_columns);
|
||||
|
||||
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(getSampleBlock().cloneWithColumns(std::move(res_columns))));
|
||||
}
|
||||
|
||||
private:
|
||||
const String name;
|
||||
};
|
||||
}
|
@ -0,0 +1,14 @@
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include <Storages/System/StorageSystemAggregateFunctionCombinators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void StorageSystemAggregateFunctionCombinators::fillData(MutableColumns & res_columns) const
|
||||
{
|
||||
const auto & combinators = AggregateFunctionCombinatorFactory::instance().getAllAggregateFunctionCombinators();
|
||||
for (const auto & pair : combinators)
|
||||
{
|
||||
res_columns[0]->insert(pair.first);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/System/IStorageSystemWithStringColumns.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
namespace DB
|
||||
{
|
||||
class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>,
|
||||
public IStorageSystemWithStringColumns<StorageSystemAggregateFunctionCombinators>
|
||||
{
|
||||
protected:
|
||||
void fillData(MutableColumns & res_columns) const override;
|
||||
|
||||
public:
|
||||
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
|
||||
|
||||
std::string getName() const override
|
||||
{
|
||||
return "SystemAggregateFunctionCombinators";
|
||||
}
|
||||
|
||||
static std::vector<String> getColumnNames()
|
||||
{
|
||||
return {"name"};
|
||||
}
|
||||
};
|
||||
}
|
91
dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp
Normal file
91
dbms/src/Storages/System/StorageSystemDataTypeFamilies.cpp
Normal file
@ -0,0 +1,91 @@
|
||||
#include <Core/Field.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Storages/System/StorageSystemDataTypeFamilies.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
String getPropertiesAsString(const DataTypePtr data_type)
|
||||
{
|
||||
std::vector<std::string> 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, ",");
|
||||
}
|
||||
ASTPtr createFakeEnumCreationAst()
|
||||
{
|
||||
String fakename = "e";
|
||||
ASTPtr name = std::make_shared<ASTLiteral>(Field(fakename.c_str(), fakename.size()));
|
||||
ASTPtr value = std::make_shared<ASTLiteral>(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;
|
||||
}
|
||||
}
|
||||
|
||||
void StorageSystemDataTypeFamilies::fillData(MutableColumns & res_columns) const
|
||||
{
|
||||
const auto & factory = DataTypeFactory::instance();
|
||||
const auto & data_types = factory.getAllDataTypes();
|
||||
for (const auto & pair : data_types)
|
||||
{
|
||||
res_columns[0]->insert(pair.first);
|
||||
|
||||
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());
|
||||
}
|
||||
else
|
||||
{
|
||||
type_ptr = factory.get(pair.first);
|
||||
}
|
||||
|
||||
res_columns[1]->insert(getPropertiesAsString(type_ptr));
|
||||
}
|
||||
catch (Exception & ex)
|
||||
{
|
||||
res_columns[1]->insert(String{"depends_on_arguments"});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
25
dbms/src/Storages/System/StorageSystemDataTypeFamilies.h
Normal file
25
dbms/src/Storages/System/StorageSystemDataTypeFamilies.h
Normal file
@ -0,0 +1,25 @@
|
||||
#pragma once
|
||||
#include <Storages/System/IStorageSystemWithStringColumns.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
namespace DB
|
||||
{
|
||||
class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper<StorageSystemDataTypeFamilies>,
|
||||
public IStorageSystemWithStringColumns<StorageSystemDataTypeFamilies>
|
||||
{
|
||||
protected:
|
||||
void fillData(MutableColumns & res_columns) const override;
|
||||
|
||||
public:
|
||||
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
|
||||
|
||||
std::string getName() const override
|
||||
{
|
||||
return "SystemTableDataTypeFamilies";
|
||||
}
|
||||
|
||||
static std::vector<String> getColumnNames()
|
||||
{
|
||||
return {"name", "properties"};
|
||||
}
|
||||
};
|
||||
}
|
30
dbms/src/Storages/System/StorageSystemFormats.cpp
Normal file
30
dbms/src/Storages/System/StorageSystemFormats.cpp
Normal file
@ -0,0 +1,30 @@
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Storages/System/StorageSystemFormats.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void StorageSystemFormats::fillData(MutableColumns & res_columns) const
|
||||
{
|
||||
const auto & formats = FormatFactory::instance().getAllFormats();
|
||||
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);
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
26
dbms/src/Storages/System/StorageSystemFormats.h
Normal file
26
dbms/src/Storages/System/StorageSystemFormats.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/System/IStorageSystemWithStringColumns.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class StorageSystemFormats : public ext::shared_ptr_helper<StorageSystemFormats>, public IStorageSystemWithStringColumns<StorageSystemFormats>
|
||||
{
|
||||
protected:
|
||||
void fillData(MutableColumns & res_columns) const override;
|
||||
public:
|
||||
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
|
||||
|
||||
std::string getName() const override
|
||||
{
|
||||
return "SystemFormats";
|
||||
}
|
||||
|
||||
static std::vector<String> getColumnNames()
|
||||
{
|
||||
return {"name", "description"};
|
||||
}
|
||||
|
||||
};
|
||||
}
|
14
dbms/src/Storages/System/StorageSystemTableFunctions.cpp
Normal file
14
dbms/src/Storages/System/StorageSystemTableFunctions.cpp
Normal file
@ -0,0 +1,14 @@
|
||||
#include <Storages/System/StorageSystemTableFunctions.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
namespace DB
|
||||
{
|
||||
void StorageSystemTableFunctions::fillData(MutableColumns & res_columns) const
|
||||
{
|
||||
const auto & functions = TableFunctionFactory::instance().getAllTableFunctions();
|
||||
for (const auto & pair : functions)
|
||||
{
|
||||
res_columns[0]->insert(pair.first);
|
||||
}
|
||||
}
|
||||
}
|
26
dbms/src/Storages/System/StorageSystemTableFunctions.h
Normal file
26
dbms/src/Storages/System/StorageSystemTableFunctions.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/System/IStorageSystemWithStringColumns.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
namespace DB
|
||||
{
|
||||
class StorageSystemTableFunctions : public ext::shared_ptr_helper<StorageSystemTableFunctions>,
|
||||
public IStorageSystemWithStringColumns<StorageSystemTableFunctions>
|
||||
{
|
||||
protected:
|
||||
void fillData(MutableColumns & res_columns) const override;
|
||||
|
||||
public:
|
||||
using IStorageSystemWithStringColumns::IStorageSystemWithStringColumns;
|
||||
|
||||
std::string getName() const override
|
||||
{
|
||||
return "SystemTableFunctions";
|
||||
}
|
||||
|
||||
static std::vector<String> getColumnNames()
|
||||
{
|
||||
return {"name"};
|
||||
}
|
||||
};
|
||||
}
|
@ -1,13 +1,16 @@
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
|
||||
#include <Storages/System/StorageSystemAggregateFunctionCombinators.h>
|
||||
#include <Storages/System/StorageSystemAsynchronousMetrics.h>
|
||||
#include <Storages/System/StorageSystemBuildOptions.h>
|
||||
#include <Storages/System/StorageSystemClusters.h>
|
||||
#include <Storages/System/StorageSystemColumns.h>
|
||||
#include <Storages/System/StorageSystemDatabases.h>
|
||||
#include <Storages/System/StorageSystemDataTypeFamilies.h>
|
||||
#include <Storages/System/StorageSystemDictionaries.h>
|
||||
#include <Storages/System/StorageSystemEvents.h>
|
||||
#include <Storages/System/StorageSystemFormats.h>
|
||||
#include <Storages/System/StorageSystemFunctions.h>
|
||||
#include <Storages/System/StorageSystemGraphite.h>
|
||||
#include <Storages/System/StorageSystemMacros.h>
|
||||
@ -23,6 +26,7 @@
|
||||
#include <Storages/System/StorageSystemReplicas.h>
|
||||
#include <Storages/System/StorageSystemReplicationQueue.h>
|
||||
#include <Storages/System/StorageSystemSettings.h>
|
||||
#include <Storages/System/StorageSystemTableFunctions.h>
|
||||
#include <Storages/System/StorageSystemTables.h>
|
||||
#include <Storages/System/StorageSystemZooKeeper.h>
|
||||
|
||||
@ -42,6 +46,10 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
system_database.attachTable("events", StorageSystemEvents::create("events"));
|
||||
system_database.attachTable("settings", StorageSystemSettings::create("settings"));
|
||||
system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options"));
|
||||
system_database.attachTable("formats", StorageSystemFormats::create("formats"));
|
||||
system_database.attachTable("table_functions", StorageSystemTableFunctions::create("table_functions"));
|
||||
system_database.attachTable("aggregate_function_combinators", StorageSystemAggregateFunctionCombinators::create("aggregate_function_combinators"));
|
||||
system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families"));
|
||||
}
|
||||
|
||||
void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
|
||||
|
@ -23,6 +23,7 @@ class TableFunctionFactory final: public ext::singleton<TableFunctionFactory>
|
||||
public:
|
||||
using Creator = std::function<TableFunctionPtr()>;
|
||||
|
||||
using TableFunctions = std::unordered_map<std::string, Creator>;
|
||||
/// Register a function by its name.
|
||||
/// No locking, you must register all functions before usage of get.
|
||||
void registerFunction(const std::string & name, Creator creator);
|
||||
@ -42,9 +43,11 @@ public:
|
||||
const std::string & name,
|
||||
const Context & context) const;
|
||||
|
||||
private:
|
||||
using TableFunctions = std::unordered_map<std::string, Creator>;
|
||||
TableFunctions getAllTableFunctions() const {
|
||||
return functions;
|
||||
}
|
||||
|
||||
private:
|
||||
TableFunctions functions;
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user