move all attaches of system tables to attach_system_tables.cpp

This commit is contained in:
proller 2017-03-14 21:39:06 +03:00 committed by alexey-milovidov
parent 227312875c
commit 25caed0c03
5 changed files with 82 additions and 74 deletions

View File

@ -0,0 +1,13 @@
#pragma once
#include <DB/Databases/IDatabase.h>
namespace DB
{
class Context;
class AsynchronousMetrics;
void attach_system_tables_server(DatabasePtr system_database, Context * global_context, bool has_zookeeper);
void attach_system_tables_local(DatabasePtr system_database);
void attach_system_tables_async(DatabasePtr system_database, AsynchronousMetrics & async_metrics);
}

View File

@ -1,42 +1,25 @@
#include "LocalServer.h"
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/Util/HelpFormatter.h>
#include <Poco/Util/OptionCallback.h>
#include <Poco/String.h>
#include <DB/Databases/DatabaseOrdinary.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemBuildOptions.h>
#include <DB/Storages/System/attach_system_tables.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Common/Exception.h>
#include <DB/Common/Macros.h>
#include <DB/Common/ConfigProcessor.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Parsers/IAST.h>
#include <common/ErrorHandlers.h>
#include <common/ApplicationServerExt.h>
#include "StatusFile.h"
@ -394,16 +377,7 @@ void LocalServer::attachSystemTables()
context->addDatabase("system", system_database);
}
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers_mt", StorageSystemNumbers::create("numbers_mt", true));
system_database->attachTable("databases", StorageSystemDatabases::create("databases"));
system_database->attachTable("tables", StorageSystemTables::create("tables"));
system_database->attachTable("columns", StorageSystemColumns::create("columns"));
system_database->attachTable("functions", StorageSystemFunctions::create("functions"));
system_database->attachTable("events", StorageSystemEvents::create("events"));
system_database->attachTable("settings", StorageSystemSettings::create("settings"));
system_database->attachTable("build_options", StorageSystemBuildOptions::create("build_options"));
attach_system_tables_local(system_database);
}

View File

@ -23,26 +23,7 @@
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Storages/MergeTree/ReshardingWorker.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Storages/System/StorageSystemAsynchronousMetrics.h>
#include <DB/Storages/System/StorageSystemBuildOptions.h>
#include <DB/Storages/System/StorageSystemClusters.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemGraphite.h>
#include <DB/Storages/System/StorageSystemMerges.h>
#include <DB/Storages/System/StorageSystemMetrics.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemParts.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemReplicas.h>
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/System/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemZooKeeper.h>
#include <DB/Storages/System/attach_system_tables.h>
#include "ConfigReloader.h"
#include "HTTPHandler.h"
#include "InterserverIOHTTPHandler.h"
@ -376,28 +357,7 @@ int Server::main(const std::vector<std::string> & args)
DatabasePtr system_database = global_context->getDatabase("system");
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers_mt", StorageSystemNumbers::create("numbers_mt", true));
system_database->attachTable("tables", StorageSystemTables::create("tables"));
system_database->attachTable("parts", StorageSystemParts::create("parts"));
system_database->attachTable("databases", StorageSystemDatabases::create("databases"));
system_database->attachTable("processes", StorageSystemProcesses::create("processes"));
system_database->attachTable("settings", StorageSystemSettings::create("settings"));
system_database->attachTable("events", StorageSystemEvents::create("events"));
system_database->attachTable("metrics", StorageSystemMetrics::create("metrics"));
system_database->attachTable("merges", StorageSystemMerges::create("merges"));
system_database->attachTable("replicas", StorageSystemReplicas::create("replicas"));
system_database->attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
system_database->attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries"));
system_database->attachTable("columns", StorageSystemColumns::create("columns"));
system_database->attachTable("functions", StorageSystemFunctions::create("functions"));
system_database->attachTable("clusters", StorageSystemClusters::create("clusters", *global_context));
system_database->attachTable("build_options", StorageSystemBuildOptions::create("build_options"));
system_database->attachTable("graphite_retentions", StorageSystemGraphite::create("graphite_retentions"));
if (has_zookeeper)
system_database->attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper"));
attach_system_tables_server(system_database, global_context.get(), has_zookeeper);
bool has_resharding_worker = false;
if (has_zookeeper && config().has("resharding"))
@ -596,8 +556,7 @@ int Server::main(const std::vector<std::string> & args)
/// This object will periodically calculate some metrics.
AsynchronousMetrics async_metrics(*global_context);
system_database->attachTable(
"asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics));
attach_system_tables_async(system_database, async_metrics);
const auto metrics_transmitter
= config().getBool("use_graphite", true) ? std::make_unique<MetricsTransmitter>(async_metrics) : nullptr;

View File

@ -1,10 +1,11 @@
#include <DB/Storages/System/StorageSystemAsynchronousMetrics.h>
#include <DB/Interpreters/AsynchronousMetrics.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumber.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/System/StorageSystemAsynchronousMetrics.h>
namespace DB

View File

@ -0,0 +1,61 @@
#include <DB/Storages/System/attach_system_tables.h>
#include <DB/Storages/System/StorageSystemAsynchronousMetrics.h>
#include <DB/Storages/System/StorageSystemBuildOptions.h>
#include <DB/Storages/System/StorageSystemClusters.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemGraphite.h>
#include <DB/Storages/System/StorageSystemMerges.h>
#include <DB/Storages/System/StorageSystemMetrics.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemParts.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemReplicas.h>
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/System/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemZooKeeper.h>
namespace DB
{
void attach_system_tables_local(DatabasePtr system_database)
{
system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
system_database->attachTable("numbers_mt", StorageSystemNumbers::create("numbers_mt", true));
system_database->attachTable("databases", StorageSystemDatabases::create("databases"));
system_database->attachTable("tables", StorageSystemTables::create("tables"));
system_database->attachTable("columns", StorageSystemColumns::create("columns"));
system_database->attachTable("functions", StorageSystemFunctions::create("functions"));
system_database->attachTable("events", StorageSystemEvents::create("events"));
system_database->attachTable("settings", StorageSystemSettings::create("settings"));
system_database->attachTable("build_options", StorageSystemBuildOptions::create("build_options"));
}
void attach_system_tables_server(DatabasePtr system_database, Context * global_context, bool has_zookeeper)
{
attach_system_tables_local(system_database);
system_database->attachTable("parts", StorageSystemParts::create("parts"));
system_database->attachTable("processes", StorageSystemProcesses::create("processes"));
system_database->attachTable("metrics", StorageSystemMetrics::create("metrics"));
system_database->attachTable("merges", StorageSystemMerges::create("merges"));
system_database->attachTable("replicas", StorageSystemReplicas::create("replicas"));
system_database->attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
system_database->attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries"));
system_database->attachTable("clusters", StorageSystemClusters::create("clusters", *global_context));
system_database->attachTable("graphite_retentions", StorageSystemGraphite::create("graphite_retentions"));
if (has_zookeeper)
system_database->attachTable("zookeeper", StorageSystemZooKeeper::create("zookeeper"));
}
void attach_system_tables_async(DatabasePtr system_database, AsynchronousMetrics & async_metrics)
{
system_database->attachTable("asynchronous_metrics", StorageSystemAsynchronousMetrics::create("asynchronous_metrics", async_metrics));
}
}