added system.macros table [#CLICKHOUSE-3591]

This commit is contained in:
Nikolai Kochetov 2018-03-12 21:38:00 +03:00
parent 4dd7f47ad5
commit 8b09e55102
4 changed files with 88 additions and 1 deletions

View File

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

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

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

View File

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