ClickHouse/dbms/src/Storages/ITableDeclaration.cpp

286 lines
9.3 KiB
C++
Raw Normal View History

#include <Storages/ITableDeclaration.h>
#include <Common/Exception.h>
#include <boost/range/join.hpp>
2012-11-30 05:24:24 +00:00
#include <sparsehash/dense_hash_map>
2014-03-09 17:36:01 +00:00
#include <sparsehash/dense_hash_set>
#include <unordered_set>
#include <sstream>
namespace DB
{
namespace ErrorCodes
{
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
extern const int COLUMN_QUERIED_MORE_THAN_ONCE;
extern const int TYPE_MISMATCH;
extern const int DUPLICATE_COLUMN;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
}
2019-05-04 16:17:11 +00:00
const ColumnsDescription & ITableDeclaration::getColumns() const
{
return columns;
}
const IndicesDescription & ITableDeclaration::getIndices() const
{
return indices;
}
void ITableDeclaration::setColumns(ColumnsDescription columns_)
{
if (columns_.getOrdinary().empty())
throw Exception("Empty list of columns passed", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
columns = std::move(columns_);
}
2019-05-02 16:07:23 +00:00
void ITableDeclaration::setIndices(IndicesDescription indices_)
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
{
indices = std::move(indices_);
}
bool ITableDeclaration::hasColumn(const String & column_name) const
2014-01-16 14:52:13 +00:00
{
2018-03-13 15:00:28 +00:00
return getColumns().hasPhysical(column_name); /// By default, we assume that there are no virtual columns in the storage.
2014-01-16 14:52:13 +00:00
}
NameAndTypePair ITableDeclaration::getColumn(const String & column_name) const
2014-01-16 14:52:13 +00:00
{
2018-03-13 15:00:28 +00:00
return getColumns().getPhysical(column_name); /// By default, we assume that there are no virtual columns in the storage.
2011-11-01 17:12:11 +00:00
}
Block ITableDeclaration::getSampleBlock() const
2011-10-31 06:37:12 +00:00
{
Block res;
2011-10-31 06:37:12 +00:00
for (const auto & col : getColumns().getAllPhysical())
res.insert({ col.type->createColumn(), col.type, col.name });
return res;
}
Block ITableDeclaration::getSampleBlockNonMaterialized() const
{
Block res;
for (const auto & col : getColumns().getOrdinary())
res.insert({ col.type->createColumn(), col.type, col.name });
2014-03-09 17:36:01 +00:00
return res;
2011-10-31 06:37:12 +00:00
}
Block ITableDeclaration::getSampleBlockForColumns(const Names & column_names) const
{
Block res;
NamesAndTypesList all_columns = getColumns().getAll();
std::unordered_map<String, DataTypePtr> columns_map;
for (const auto & elem : all_columns)
columns_map.emplace(elem.name, elem.type);
for (const auto & name : column_names)
{
auto it = columns_map.find(name);
if (it != columns_map.end())
{
res.insert({ it->second->createColumn(), it->second, it->first });
}
else
{
/// Virtual columns.
NameAndTypePair elem = getColumn(name);
res.insert({ elem.type->createColumn(), elem.type, elem.name });
}
}
return res;
}
static std::string listOfColumns(const NamesAndTypesList & available_columns)
2012-11-30 05:24:24 +00:00
{
std::stringstream s;
for (auto it = available_columns.begin(); it != available_columns.end(); ++it)
{
if (it != available_columns.begin())
s << ", ";
s << it->name;
}
return s.str();
2012-11-30 05:24:24 +00:00
}
using NamesAndTypesMap = GOOGLE_NAMESPACE::dense_hash_map<StringRef, const IDataType *, StringRefHash>;
2012-11-30 05:24:24 +00:00
static NamesAndTypesMap & getColumnsMapImpl(NamesAndTypesMap & res) { return res; }
2012-11-30 05:24:24 +00:00
template <typename Arg, typename... Args>
static NamesAndTypesMap & getColumnsMapImpl(NamesAndTypesMap & res, const Arg & arg, const Args &... args)
{
static_assert(std::is_same_v<Arg, NamesAndTypesList>, "getColumnsMap requires arguments of type NamesAndTypesList");
for (const auto & column : arg)
res.insert({column.name, column.type.get()});
return getColumnsMapImpl(res, args...);
}
2012-11-30 05:24:24 +00:00
template <typename... Args>
static NamesAndTypesMap getColumnsMap(const Args &... args)
2012-11-30 05:24:24 +00:00
{
NamesAndTypesMap res;
res.set_empty_key(StringRef());
2012-11-30 05:24:24 +00:00
return getColumnsMapImpl(res, args...);
2012-11-30 05:24:24 +00:00
}
void ITableDeclaration::check(const Names & column_names) const
{
2018-03-13 15:00:28 +00:00
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
2014-03-09 17:36:01 +00:00
if (column_names.empty())
throw Exception("Empty list of columns queried. There are columns: " + listOfColumns(available_columns),
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
const auto columns_map = getColumnsMap(available_columns);
2012-11-30 05:24:24 +00:00
using UniqueStrings = GOOGLE_NAMESPACE::dense_hash_set<StringRef, StringRefHash>;
UniqueStrings unique_names;
unique_names.set_empty_key(StringRef());
for (const auto & name : column_names)
{
if (columns_map.end() == columns_map.find(name))
throw Exception("There is no column with name " + name + " in table. There are columns: " + listOfColumns(available_columns),
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
if (unique_names.end() != unique_names.find(name))
throw Exception("Column " + name + " queried more than once",
ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
unique_names.insert(name);
}
}
void ITableDeclaration::check(const NamesAndTypesList & provided_columns) const
2014-07-14 14:07:47 +00:00
{
2018-03-13 15:00:28 +00:00
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
const auto columns_map = getColumnsMap(available_columns);
using UniqueStrings = GOOGLE_NAMESPACE::dense_hash_set<StringRef, StringRefHash>;
UniqueStrings unique_names;
unique_names.set_empty_key(StringRef());
for (const NameAndTypePair & column : provided_columns)
{
NamesAndTypesMap::const_iterator it = columns_map.find(column.name);
if (columns_map.end() == it)
throw Exception("There is no column with name " + column.name + ". There are columns: "
+ listOfColumns(available_columns), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
if (!column.type->equals(*it->second))
throw Exception("Type mismatch for column " + column.name + ". Column has type "
+ it->second->getName() + ", got type " + column.type->getName(), ErrorCodes::TYPE_MISMATCH);
if (unique_names.end() != unique_names.find(column.name))
throw Exception("Column " + column.name + " queried more than once",
ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
unique_names.insert(column.name);
}
2014-07-14 14:07:47 +00:00
}
void ITableDeclaration::check(const NamesAndTypesList & provided_columns, const Names & column_names) const
2014-07-14 14:07:47 +00:00
{
2018-03-13 15:00:28 +00:00
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
const auto available_columns_map = getColumnsMap(available_columns);
const NamesAndTypesMap & provided_columns_map = getColumnsMap(provided_columns);
if (column_names.empty())
throw Exception("Empty list of columns queried. There are columns: " + listOfColumns(available_columns),
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
using UniqueStrings = GOOGLE_NAMESPACE::dense_hash_set<StringRef, StringRefHash>;
UniqueStrings unique_names;
unique_names.set_empty_key(StringRef());
for (const String & name : column_names)
{
NamesAndTypesMap::const_iterator it = provided_columns_map.find(name);
if (provided_columns_map.end() == it)
continue;
NamesAndTypesMap::const_iterator jt = available_columns_map.find(name);
if (available_columns_map.end() == jt)
throw Exception("There is no column with name " + name + ". There are columns: "
+ listOfColumns(available_columns), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
if (!it->second->equals(*jt->second))
throw Exception("Type mismatch for column " + name + ". Column has type "
+ jt->second->getName() + ", got type " + it->second->getName(), ErrorCodes::TYPE_MISMATCH);
if (unique_names.end() != unique_names.find(name))
throw Exception("Column " + name + " queried more than once",
ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
unique_names.insert(name);
}
2014-07-14 14:07:47 +00:00
}
void ITableDeclaration::check(const Block & block, bool need_all) const
{
2018-03-13 15:00:28 +00:00
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
const auto columns_map = getColumnsMap(available_columns);
NameSet names_in_block;
2018-01-23 18:49:15 +00:00
block.checkNumberOfRows();
2018-01-23 18:49:15 +00:00
for (const auto & column : block)
{
if (names_in_block.count(column.name))
throw Exception("Duplicate column " + column.name + " in block",
ErrorCodes::DUPLICATE_COLUMN);
names_in_block.insert(column.name);
NamesAndTypesMap::const_iterator it = columns_map.find(column.name);
if (columns_map.end() == it)
throw Exception("There is no column with name " + column.name + ". There are columns: "
+ listOfColumns(available_columns), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
if (!column.type->equals(*it->second))
throw Exception("Type mismatch for column " + column.name + ". Column has type "
+ it->second->getName() + ", got type " + column.type->getName(), ErrorCodes::TYPE_MISMATCH);
}
if (need_all && names_in_block.size() < columns_map.size())
{
for (NamesAndTypesList::const_iterator it = available_columns.begin(); it != available_columns.end(); ++it)
{
if (!names_in_block.count(it->name))
throw Exception("Expected column " + it->name, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
}
}
}
ITableDeclaration::ITableDeclaration(ColumnsDescription columns_)
{
setColumns(std::move(columns_));
}
}