mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
added system.macros table [#CLICKHOUSE-3591]
This commit is contained in:
parent
4dd7f47ad5
commit
8b09e55102
@ -21,9 +21,10 @@ public:
|
||||
*/
|
||||
String expand(const String & s, size_t level = 0) const;
|
||||
|
||||
private:
|
||||
using MacroMap = std::map<String, String>;
|
||||
const MacroMap getMacroMap() const { return macros; }
|
||||
|
||||
private:
|
||||
MacroMap macros;
|
||||
};
|
||||
|
||||
|
48
dbms/src/Storages/System/StorageSystemMacros.cpp
Normal file
48
dbms/src/Storages/System/StorageSystemMacros.cpp
Normal file
@ -0,0 +1,48 @@
|
||||
#include <Common/Macros.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Storages/System/StorageSystemMacros.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
StorageSystemMacros::StorageSystemMacros(const std::string & name_)
|
||||
: name(name_)
|
||||
{
|
||||
columns = NamesAndTypesList{
|
||||
{"macro", std::make_shared<DataTypeString>()},
|
||||
{"substitution", std::make_shared<DataTypeString>()},
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
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*/)
|
||||
{
|
||||
check(column_names);
|
||||
processed_stage = QueryProcessingStage::FetchColumns;
|
||||
|
||||
MutableColumns res_columns = getSampleBlock().cloneEmptyColumns();
|
||||
|
||||
auto macros = context.getMacros();
|
||||
|
||||
for (const auto & macro : macros->getMacroMap())
|
||||
{
|
||||
res_columns[0]->insert(macro.first);
|
||||
res_columns[1]->insert(macro.second);
|
||||
}
|
||||
|
||||
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(getSampleBlock().cloneWithColumns(std::move(res_columns))));
|
||||
}
|
||||
|
||||
|
||||
}
|
36
dbms/src/Storages/System/StorageSystemMacros.h
Normal file
36
dbms/src/Storages/System/StorageSystemMacros.h
Normal file
@ -0,0 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/** Implements `databases` system table, which allows you to get information about all databases.
|
||||
*/
|
||||
class StorageSystemMacros : public ext::shared_ptr_helper<StorageSystemMacros>, public IStorage
|
||||
{
|
||||
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;
|
||||
|
||||
protected:
|
||||
StorageSystemMacros(const std::string & name_);
|
||||
};
|
||||
|
||||
}
|
@ -10,6 +10,7 @@
|
||||
#include <Storages/System/StorageSystemEvents.h>
|
||||
#include <Storages/System/StorageSystemFunctions.h>
|
||||
#include <Storages/System/StorageSystemGraphite.h>
|
||||
#include <Storages/System/StorageSystemMacros.h>
|
||||
#include <Storages/System/StorageSystemMerges.h>
|
||||
#include <Storages/System/StorageSystemMetrics.h>
|
||||
#include <Storages/System/StorageSystemModels.h>
|
||||
@ -56,6 +57,7 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
|
||||
system_database.attachTable("models", StorageSystemModels::create("models"));
|
||||
system_database.attachTable("clusters", StorageSystemClusters::create("clusters"));
|
||||
system_database.attachTable("graphite_retentions", StorageSystemGraphite::create("graphite_retentions"));
|
||||
system_database.attachTable("macros", StorageSystemMacros::create("macros"));
|
||||
|
||||
if (has_zookeeper)
|
||||
system_database.attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper"));
|
||||
|
Loading…
Reference in New Issue
Block a user