ClickHouse/dbms/src/Databases/DatabaseDictionary.cpp

204 lines
6.5 KiB
C++
Raw Normal View History

2017-06-22 15:44:19 +00:00
#include <Databases/DatabaseDictionary.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Storages/StorageDictionary.h>
#include <common/logger_useful.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
2019-02-10 17:40:52 +00:00
#include <Parsers/IAST.h>
2017-06-22 15:44:19 +00:00
namespace DB
{
namespace ErrorCodes
{
2018-03-23 19:56:24 +00:00
extern const int TABLE_ALREADY_EXISTS;
extern const int UNKNOWN_TABLE;
extern const int LOGICAL_ERROR;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int SYNTAX_ERROR;
2017-06-22 15:44:19 +00:00
}
DatabaseDictionary::DatabaseDictionary(const String & name_)
: name(name_),
log(&Logger::get("DatabaseDictionary(" + name + ")"))
{
}
2017-12-01 20:21:35 +00:00
void DatabaseDictionary::loadTables(Context &, ThreadPool *, bool)
2017-06-22 15:44:19 +00:00
{
}
2017-06-22 15:44:19 +00:00
Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name)
{
Tables tables;
ExternalLoader::Loadables loadables;
if (filter_by_name)
{
/// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them.
loadables = context.getExternalDictionaries().loadAndGet(filter_by_name);
}
else
2017-06-22 15:44:19 +00:00
{
/// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case.
loadables = context.getExternalDictionaries().getCurrentlyLoadedObjects();
2017-06-22 15:44:19 +00:00
}
for (const auto & loadable : loadables)
{
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(loadable);
auto dict_name = dict_ptr->getName();
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, context, true, dict_name);
}
return tables;
2017-06-22 15:44:19 +00:00
}
bool DatabaseDictionary::isTableExist(
const Context & context,
const String & table_name) const
2017-06-22 15:44:19 +00:00
{
return context.getExternalDictionaries().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
2017-06-22 15:44:19 +00:00
}
StoragePtr DatabaseDictionary::tryGetTable(
const Context & context,
const String & table_name) const
2017-06-22 15:44:19 +00:00
{
auto dict_ptr = context.getExternalDictionaries().tryGetDictionary(table_name);
if (dict_ptr)
{
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
return StorageDictionary::create(table_name, ColumnsDescription{columns}, context, true, table_name);
}
return {};
2017-06-22 15:44:19 +00:00
}
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context, const FilterByNameFunction & filter_by_name)
2017-06-22 15:44:19 +00:00
{
return std::make_unique<DatabaseSnapshotIterator>(listTables(context, filter_by_name));
2017-06-22 15:44:19 +00:00
}
bool DatabaseDictionary::empty(const Context & context) const
2017-06-22 15:44:19 +00:00
{
return context.getExternalDictionaries().getNumberOfNames() == 0;
2017-06-22 15:44:19 +00:00
}
2017-12-01 20:21:35 +00:00
StoragePtr DatabaseDictionary::detachTable(const String & /*table_name*/)
2017-06-22 15:44:19 +00:00
{
throw Exception("DatabaseDictionary: detachTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
2017-12-01 20:21:35 +00:00
void DatabaseDictionary::attachTable(const String & /*table_name*/, const StoragePtr & /*table*/)
2017-06-22 15:44:19 +00:00
{
throw Exception("DatabaseDictionary: attachTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
void DatabaseDictionary::createTable(
const Context &,
const String &,
const StoragePtr &,
const ASTPtr &)
2017-06-22 15:44:19 +00:00
{
throw Exception("DatabaseDictionary: createTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
2017-06-22 15:44:19 +00:00
}
void DatabaseDictionary::removeTable(
const Context &,
const String &)
2017-06-22 15:44:19 +00:00
{
throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
2017-06-22 15:44:19 +00:00
}
void DatabaseDictionary::renameTable(
2017-12-01 20:21:35 +00:00
const Context &,
const String &,
IDatabase &,
const String &)
2017-06-22 15:44:19 +00:00
{
throw Exception("DatabaseDictionary: renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
void DatabaseDictionary::alterTable(
2017-12-01 20:21:35 +00:00
const Context &,
const String &,
const ColumnsDescription &,
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
const IndicesDescription &,
2017-12-01 20:21:35 +00:00
const ASTModifier &)
{
throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
time_t DatabaseDictionary::getTableMetadataModificationTime(
2017-12-01 20:21:35 +00:00
const Context &,
const String &)
2017-06-22 15:44:19 +00:00
{
return static_cast<time_t>(0);
}
2018-03-23 19:56:24 +00:00
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
const String & table_name, bool throw_on_error) const
2017-06-22 15:44:19 +00:00
{
String query;
{
WriteBufferFromString buffer(query);
2018-03-23 19:56:24 +00:00
const auto & dictionaries = context.getExternalDictionaries();
auto dictionary = throw_on_error ? dictionaries.getDictionary(table_name)
: dictionaries.tryGetDictionary(table_name);
auto names_and_types = StorageDictionary::getNamesAndTypes(dictionary->getStructure());
buffer << "CREATE TABLE " << backQuoteIfNeed(name) << '.' << backQuoteIfNeed(table_name) << " (";
buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types.begin(), names_and_types.end());
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
}
ParserCreateQuery parser;
2018-03-23 19:56:24 +00:00
const char * pos = query.data();
std::string error_message;
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message,
2018-04-16 15:39:12 +00:00
/* hilite = */ false, "", /* allow_multi_statements = */ false, 0);
2018-03-23 19:56:24 +00:00
if (!ast && throw_on_error)
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
return ast;
}
ASTPtr DatabaseDictionary::getCreateTableQuery(const Context & context, const String & table_name) const
{
return getCreateTableQueryImpl(context, table_name, true);
}
ASTPtr DatabaseDictionary::tryGetCreateTableQuery(const Context & context, const String & table_name) const
{
2018-03-23 19:56:24 +00:00
return getCreateTableQueryImpl(context, table_name, false);
}
ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & /*context*/) const
2017-06-22 15:44:19 +00:00
{
String query;
{
WriteBufferFromString buffer(query);
buffer << "CREATE DATABASE " << backQuoteIfNeed(name) << " ENGINE = Dictionary";
}
ParserCreateQuery parser;
2018-04-16 15:39:12 +00:00
return parseQuery(parser, query.data(), query.data() + query.size(), "", 0);
2017-06-22 15:44:19 +00:00
}
void DatabaseDictionary::shutdown()
{
}
String DatabaseDictionary::getDatabaseName() const
{
return name;
}
2017-06-22 15:44:19 +00:00
}