CLICKHOUSE-3772: Add system tables formats, data_type_families, table_functions, aggregate_function_combinators

This commit is contained in:
alesapin 2018-07-20 13:00:56 +03:00
parent d0d61e4ea4
commit ec156e9f2b
14 changed files with 330 additions and 2 deletions

View File

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

View File

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

View File

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

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

View File

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

View File

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

View 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"});
}
}
}
}

View 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"};
}
};
}

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

View 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"};
}
};
}

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

View 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"};
}
};
}

View File

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

View File

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