dbms:dummy system.dictionaries and dictionary exception logging. [#METR-13298]

This commit is contained in:
Andrey Mironov 2015-03-24 12:46:07 +03:00
parent b332a14f1f
commit e521611c98
5 changed files with 130 additions and 14 deletions

View File

@ -38,6 +38,8 @@ class ExternalDictionaries
private:
static const auto check_period_sec = 5;
friend class StorageSystemDictionaries;
mutable std::mutex dictionaries_mutex;
using dictionary_ptr_t = std::shared_ptr<MultiVersion<IDictionary>>;

View File

@ -0,0 +1,34 @@
#pragma once
#include <DB/Storages/IStorage.h>
namespace DB
{
class StorageSystemDictionaries : public IStorage
{
public:
static StoragePtr create(const std::string & name);
std::string getName() const override { return "SystemDictionaries"; }
std::string getTableName() const override { return name; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
private:
const std::string name;
NamesAndTypesList columns;
StorageSystemDictionaries(const std::string & name);
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
};
}

View File

@ -44,6 +44,8 @@ void ExternalDictionaries::reloadImpl()
/// periodic update
for (auto & dictionary : dictionaries)
{
const auto & name = dictionary.first;
try
{
auto current = dictionary.second.first->get();
@ -76,21 +78,33 @@ void ExternalDictionaries::reloadImpl()
dictionary.second.first->set(new_version.release());
}
}
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first
<< "'! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first
<< "'! You must resolve this manually. " << e.what());
/// erase stored exception on success
stored_exceptions.erase(name);
}
catch (...)
{
LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first
<< "'! You must resolve this manually.");
stored_exceptions.emplace(name, std::current_exception());
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << name
<< "'! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << name
<< "'! You must resolve this manually. " << e.what());
}
catch (...)
{
LOG_ERROR(log, "Cannot update external dictionary '" << name
<< "'! You must resolve this manually.");
}
}
}
}
@ -113,8 +127,6 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path)
const auto last_modified = config_file.getLastModified();
if (last_modified > config_last_modified)
{
stored_exceptions.clear();
/// definitions of dictionaries may have changed, recreate all of them
config_last_modified = last_modified;
@ -179,6 +191,9 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path)
}
else
it->second.first->set(dict_ptr.release());
/// erase stored exception on success
stored_exceptions.erase(name);
}
catch (...)
{

View File

@ -22,6 +22,7 @@
#include <DB/Storages/StorageSystemSettings.h>
#include <DB/Storages/StorageSystemZooKeeper.h>
#include <DB/Storages/StorageSystemReplicas.h>
#include <DB/Storages/StorageSystemDictionaries.h>
#include <DB/IO/copyData.h>
#include <DB/IO/LimitReadBuffer.h>
@ -528,6 +529,7 @@ int Server::main(const std::vector<std::string> & args)
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
global_context->addTable("system", "merges", StorageSystemMerges::create("merges"));
global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas"));
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
if (has_zookeeper)
global_context->addTable("system", "zookeeper", StorageSystemZooKeeper::create("zookeeper"));

View File

@ -0,0 +1,63 @@
#include <DB/Storages/StorageSystemDictionaries.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Columns/ColumnString.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Interpreters/Context.h>
#include <DB/Dictionaries/IDictionary.h>
#include <mutex>
namespace DB
{
StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
: name{name},
columns{
{ "name", new DataTypeString },
{ "type", new DataTypeString },
{ "origin", new DataTypeString },
}
{
}
StoragePtr StorageSystemDictionaries::create(const std::string & name)
{
return (new StorageSystemDictionaries{name})->thisPtr();
}
BlockInputStreams StorageSystemDictionaries::read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
const unsigned)
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
ColumnWithNameAndType col_name{new ColumnString, new DataTypeString, "name"};
ColumnWithNameAndType col_type{new ColumnString, new DataTypeString, "type"};
ColumnWithNameAndType col_origin{new ColumnString, new DataTypeString, "origin"};
const auto & external_dictionaries = context.getExternalDictionaries();
const std::lock_guard<std::mutex> lock{external_dictionaries.dictionaries_mutex};
for (const auto & dict_info : external_dictionaries.dictionaries)
{
col_name.column->insert(dict_info.first);
col_type.column->insert(dict_info.second.first->get()->getTypeName());
col_origin.column->insert(dict_info.second.second);
}
Block block{
col_name,
col_type,
col_origin
};
return BlockInputStreams{1, new OneBlockInputStream{block}};
}
}