mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-20 14:42:02 +00:00
Merge
This commit is contained in:
commit
52fc1d3560
@ -20,6 +20,8 @@ namespace DB
|
|||||||
* Позволяет вставлять, удалять столбцы в любом порядке, менять порядок столбцов.
|
* Позволяет вставлять, удалять столбцы в любом порядке, менять порядок столбцов.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
|
||||||
class Block
|
class Block
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -54,6 +56,7 @@ public:
|
|||||||
void insert(size_t position, const ColumnWithNameAndType & elem);
|
void insert(size_t position, const ColumnWithNameAndType & elem);
|
||||||
/// вставить столбец в конец
|
/// вставить столбец в конец
|
||||||
void insert(const ColumnWithNameAndType & elem);
|
void insert(const ColumnWithNameAndType & elem);
|
||||||
|
void insertDefault(const String & name, const DataTypePtr & type);
|
||||||
/// вставить столбец в конец, если столбца с таким именем ещё нет
|
/// вставить столбец в конец, если столбца с таким именем ещё нет
|
||||||
void insertUnique(const ColumnWithNameAndType & elem);
|
void insertUnique(const ColumnWithNameAndType & elem);
|
||||||
/// удалить столбец в заданной позиции
|
/// удалить столбец в заданной позиции
|
||||||
@ -61,7 +64,7 @@ public:
|
|||||||
/// удалить столбец с заданным именем
|
/// удалить столбец с заданным именем
|
||||||
void erase(const String & name);
|
void erase(const String & name);
|
||||||
/// Добавляет в блок недостающие столбцы со значениями по-умолчанию
|
/// Добавляет в блок недостающие столбцы со значениями по-умолчанию
|
||||||
void addDefaults(NamesAndTypesListPtr required_columns);
|
void addDefaults(const NamesAndTypesList & required_columns);
|
||||||
|
|
||||||
ColumnWithNameAndType & getByPosition(size_t position);
|
ColumnWithNameAndType & getByPosition(size_t position);
|
||||||
const ColumnWithNameAndType & getByPosition(size_t position) const;
|
const ColumnWithNameAndType & getByPosition(size_t position) const;
|
||||||
|
@ -3,8 +3,10 @@
|
|||||||
#include <Poco/SharedPtr.h>
|
#include <Poco/SharedPtr.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
|
#include <DB/Interpreters/evaluateMissingDefaults.h>
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
|
|
||||||
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -19,12 +21,20 @@ class AddingDefaultBlockInputStream : public IProfilingBlockInputStream
|
|||||||
public:
|
public:
|
||||||
AddingDefaultBlockInputStream(
|
AddingDefaultBlockInputStream(
|
||||||
BlockInputStreamPtr input_,
|
BlockInputStreamPtr input_,
|
||||||
NamesAndTypesListPtr required_columns_)
|
NamesAndTypesListPtr required_columns_,
|
||||||
: required_columns(required_columns_)
|
const ColumnDefaults & column_defaults_,
|
||||||
|
const Context & context_)
|
||||||
|
: required_columns(required_columns_),
|
||||||
|
column_defaults(column_defaults_), context(context_)
|
||||||
{
|
{
|
||||||
children.push_back(input_);
|
children.push_back(input_);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
AddingDefaultBlockInputStream(BlockInputStreamPtr input_, NamesAndTypesListPtr required_columns_, const Context & context_)
|
||||||
|
: AddingDefaultBlockInputStream{input_, required_columns, {}, context}
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
String getName() const override { return "AddingDefaultBlockInputStream"; }
|
String getName() const override { return "AddingDefaultBlockInputStream"; }
|
||||||
|
|
||||||
String getID() const override
|
String getID() const override
|
||||||
@ -45,12 +55,15 @@ protected:
|
|||||||
Block res = children.back()->read();
|
Block res = children.back()->read();
|
||||||
if (!res)
|
if (!res)
|
||||||
return res;
|
return res;
|
||||||
res.addDefaults(required_columns);
|
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
||||||
|
res.addDefaults(*required_columns);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
NamesAndTypesListPtr required_columns;
|
NamesAndTypesListPtr required_columns;
|
||||||
|
const ColumnDefaults & column_defaults;
|
||||||
|
Context context;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -5,6 +5,9 @@
|
|||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Columns/ColumnConst.h>
|
||||||
|
|
||||||
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
|
#include <DB/Interpreters/evaluateMissingDefaults.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -19,15 +22,25 @@ class AddingDefaultBlockOutputStream : public IBlockOutputStream
|
|||||||
public:
|
public:
|
||||||
AddingDefaultBlockOutputStream(
|
AddingDefaultBlockOutputStream(
|
||||||
BlockOutputStreamPtr output_,
|
BlockOutputStreamPtr output_,
|
||||||
NamesAndTypesListPtr required_columns_)
|
NamesAndTypesListPtr required_columns_,
|
||||||
: output(output_), required_columns(required_columns_)
|
const ColumnDefaults & column_defaults_,
|
||||||
|
const Context & context_)
|
||||||
|
: output(output_), required_columns(required_columns_),
|
||||||
|
column_defaults(column_defaults_), context(context_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
AddingDefaultBlockOutputStream(BlockOutputStreamPtr output_, NamesAndTypesListPtr required_columns_, const Context & context_)
|
||||||
|
: AddingDefaultBlockOutputStream{output_, required_columns_, {}, context_}
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void write(const Block & block) override
|
void write(const Block & block) override
|
||||||
{
|
{
|
||||||
Block res = block;
|
Block res = block;
|
||||||
res.addDefaults(required_columns);
|
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
||||||
|
res.addDefaults(*required_columns);
|
||||||
output->write(res);
|
output->write(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -39,6 +52,8 @@ public:
|
|||||||
private:
|
private:
|
||||||
BlockOutputStreamPtr output;
|
BlockOutputStreamPtr output;
|
||||||
NamesAndTypesListPtr required_columns;
|
NamesAndTypesListPtr required_columns;
|
||||||
|
const ColumnDefaults & column_defaults;
|
||||||
|
Context context;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
37
dbms/include/DB/DataStreams/MaterializingBlockOutputStream.h
Normal file
37
dbms/include/DB/DataStreams/MaterializingBlockOutputStream.h
Normal file
@ -0,0 +1,37 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Columns/ColumnConst.h>
|
||||||
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
#include <statdaemons/ext/range.hpp>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/** Преобразует столбцы-константы в полноценные столбцы ("материализует" их).
|
||||||
|
*/
|
||||||
|
class MaterializingBlockOutputStream : public IBlockOutputStream
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
MaterializingBlockOutputStream(const BlockOutputStreamPtr & output) : output{output}
|
||||||
|
{}
|
||||||
|
|
||||||
|
void write(const Block & original_block) override
|
||||||
|
{
|
||||||
|
/// copy block to get rid of const
|
||||||
|
auto block = original_block;
|
||||||
|
|
||||||
|
for (const auto i : ext::range(0, block.columns()))
|
||||||
|
{
|
||||||
|
ColumnPtr & col = block.getByPosition(i).column;
|
||||||
|
if (col->isConst())
|
||||||
|
col = dynamic_cast<IColumnConst &>(*col).convertToFullColumn();
|
||||||
|
}
|
||||||
|
|
||||||
|
output->write(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
BlockOutputStreamPtr output;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,39 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
/// Throws exception on encountering prohibited column in block
|
||||||
|
class ProhibitColumnsBlockOutputStream : public IBlockOutputStream
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
ProhibitColumnsBlockOutputStream(const BlockOutputStreamPtr & output, const NamesAndTypesList & columns)
|
||||||
|
: output{output}, columns{columns}
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
void write(const Block & block) override
|
||||||
|
{
|
||||||
|
for (const auto & column : columns)
|
||||||
|
if (block.has(column.name))
|
||||||
|
throw Exception{"Cannot insert column " + column.name, ErrorCodes::ILLEGAL_COLUMN};
|
||||||
|
|
||||||
|
output->write(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
void flush() override { output->flush(); }
|
||||||
|
|
||||||
|
void writePrefix() override { output->writePrefix(); }
|
||||||
|
void writeSuffix() override { output->writeSuffix(); }
|
||||||
|
|
||||||
|
BlockOutputStreamPtr output;
|
||||||
|
NamesAndTypesList columns;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -25,9 +25,10 @@ public:
|
|||||||
|
|
||||||
FunctionPtr get(const String & name, const Context & context) const;
|
FunctionPtr get(const String & name, const Context & context) const;
|
||||||
|
|
||||||
void registerFunction(const String & name, Creator creator)
|
template <typename F> void registerFunction()
|
||||||
{
|
{
|
||||||
functions[name] = creator;
|
static_assert(std::is_same<decltype(&F::create), Creator>::value, "F::create has incorrect type");
|
||||||
|
functions[F::name] = &F::create;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -367,6 +367,10 @@ struct BinaryOperationTraits
|
|||||||
template <template <typename, typename> class Op, typename Name>
|
template <template <typename, typename> class Op, typename Name>
|
||||||
class FunctionBinaryArithmetic : public IFunction
|
class FunctionBinaryArithmetic : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionBinaryArithmetic; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
/// Overload for InvalidType
|
/// Overload for InvalidType
|
||||||
template <typename ResultDataType,
|
template <typename ResultDataType,
|
||||||
@ -585,7 +589,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -641,6 +645,10 @@ public:
|
|||||||
template <template <typename> class Op, typename Name>
|
template <template <typename> class Op, typename Name>
|
||||||
class FunctionUnaryArithmetic : public IFunction
|
class FunctionUnaryArithmetic : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionUnaryArithmetic; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
template <typename T0>
|
template <typename T0>
|
||||||
bool checkType(const DataTypes & arguments, DataTypePtr & result) const
|
bool checkType(const DataTypes & arguments, DataTypePtr & result) const
|
||||||
@ -690,7 +698,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -739,19 +747,19 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NamePlus { static const char * get() { return "plus"; } };
|
struct NamePlus { static constexpr auto name = "plus"; };
|
||||||
struct NameMinus { static const char * get() { return "minus"; } };
|
struct NameMinus { static constexpr auto name = "minus"; };
|
||||||
struct NameMultiply { static const char * get() { return "multiply"; } };
|
struct NameMultiply { static constexpr auto name = "multiply"; };
|
||||||
struct NameDivideFloating { static const char * get() { return "divide"; } };
|
struct NameDivideFloating { static constexpr auto name = "divide"; };
|
||||||
struct NameDivideIntegral { static const char * get() { return "intDiv"; } };
|
struct NameDivideIntegral { static constexpr auto name = "intDiv"; };
|
||||||
struct NameModulo { static const char * get() { return "modulo"; } };
|
struct NameModulo { static constexpr auto name = "modulo"; };
|
||||||
struct NameNegate { static const char * get() { return "negate"; } };
|
struct NameNegate { static constexpr auto name = "negate"; };
|
||||||
struct NameBitAnd { static const char * get() { return "bitAnd"; } };
|
struct NameBitAnd { static constexpr auto name = "bitAnd"; };
|
||||||
struct NameBitOr { static const char * get() { return "bitOr"; } };
|
struct NameBitOr { static constexpr auto name = "bitOr"; };
|
||||||
struct NameBitXor { static const char * get() { return "bitXor"; } };
|
struct NameBitXor { static constexpr auto name = "bitXor"; };
|
||||||
struct NameBitNot { static const char * get() { return "bitNot"; } };
|
struct NameBitNot { static constexpr auto name = "bitNot"; };
|
||||||
struct NameBitShiftLeft { static const char * get() { return "bitShiftLeft"; } };
|
struct NameBitShiftLeft { static constexpr auto name = "bitShiftLeft"; };
|
||||||
struct NameBitShiftRight { static const char * get() { return "bitShiftRight"; } };
|
struct NameBitShiftRight { static constexpr auto name = "bitShiftRight"; };
|
||||||
|
|
||||||
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
||||||
typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
|
typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
|
||||||
|
@ -41,11 +41,15 @@ namespace DB
|
|||||||
|
|
||||||
class FunctionArray : public IFunction
|
class FunctionArray : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "array";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionArray; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "array";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T0, typename T1>
|
template <typename T0, typename T1>
|
||||||
@ -384,6 +388,10 @@ struct ArrayElementStringImpl
|
|||||||
|
|
||||||
class FunctionArrayElement : public IFunction
|
class FunctionArrayElement : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "arrayElement";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionArrayElement; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
template <typename T>
|
template <typename T>
|
||||||
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||||
@ -592,7 +600,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "arrayElement";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -755,6 +763,10 @@ struct ArrayIndexStringImpl
|
|||||||
template <typename IndexConv, typename Name>
|
template <typename IndexConv, typename Name>
|
||||||
class FunctionArrayIndex : public IFunction
|
class FunctionArrayIndex : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionArrayIndex; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
typedef ColumnVector<typename IndexConv::ResultType> ResultColumnType;
|
typedef ColumnVector<typename IndexConv::ResultType> ResultColumnType;
|
||||||
|
|
||||||
@ -842,7 +854,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -893,10 +905,13 @@ public:
|
|||||||
class FunctionArrayEnumerate : public IFunction
|
class FunctionArrayEnumerate : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "arrayEnumerate";
|
||||||
|
static IFunction * create (const Context & context) { return new FunctionArrayEnumerate; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "arrayEnumerate";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -963,10 +978,13 @@ public:
|
|||||||
class FunctionArrayEnumerateUniq : public IFunction
|
class FunctionArrayEnumerateUniq : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "arrayEnumerateUniq";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionArrayEnumerateUniq; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "arrayEnumerateUniq";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1202,11 +1220,15 @@ template <> struct DataTypeToName<DataTypeDate> { static std::string get() { ret
|
|||||||
template <> struct DataTypeToName<DataTypeDateTime> { static std::string get() { return "DateTime"; } };
|
template <> struct DataTypeToName<DataTypeDateTime> { static std::string get() { return "DateTime"; } };
|
||||||
|
|
||||||
template <typename DataType>
|
template <typename DataType>
|
||||||
struct EmptyArray : public IFunction
|
struct FunctionEmptyArray : public IFunction
|
||||||
{
|
{
|
||||||
|
static constexpr auto base_name = "emptyArray";
|
||||||
|
static const String name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionEmptyArray; }
|
||||||
|
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "emptyArray" + DataTypeToName<DataType>::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
@ -1230,28 +1252,31 @@ struct EmptyArray : public IFunction
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
template <typename DataType>
|
||||||
|
const String FunctionEmptyArray<DataType>::name = FunctionEmptyArray::base_name + DataTypeToName<DataType>::get();
|
||||||
|
|
||||||
struct NameHas { static const char * get() { return "has"; } };
|
|
||||||
struct NameIndexOf { static const char * get() { return "indexOf"; } };
|
struct NameHas { static constexpr auto name = "has"; };
|
||||||
struct NameCountEqual { static const char * get() { return "countEqual"; } };
|
struct NameIndexOf { static constexpr auto name = "indexOf"; };
|
||||||
|
struct NameCountEqual { static constexpr auto name = "countEqual"; };
|
||||||
|
|
||||||
typedef FunctionArrayIndex<IndexToOne, NameHas> FunctionHas;
|
typedef FunctionArrayIndex<IndexToOne, NameHas> FunctionHas;
|
||||||
typedef FunctionArrayIndex<IndexIdentity, NameIndexOf> FunctionIndexOf;
|
typedef FunctionArrayIndex<IndexIdentity, NameIndexOf> FunctionIndexOf;
|
||||||
typedef FunctionArrayIndex<IndexCount, NameCountEqual> FunctionCountEqual;
|
typedef FunctionArrayIndex<IndexCount, NameCountEqual> FunctionCountEqual;
|
||||||
|
|
||||||
using FunctionEmptyArrayUInt8 = EmptyArray<DataTypeUInt8>;
|
using FunctionEmptyArrayUInt8 = FunctionEmptyArray<DataTypeUInt8>;
|
||||||
using FunctionEmptyArrayUInt16 = EmptyArray<DataTypeUInt16>;
|
using FunctionEmptyArrayUInt16 = FunctionEmptyArray<DataTypeUInt16>;
|
||||||
using FunctionEmptyArrayUInt32 = EmptyArray<DataTypeUInt32>;
|
using FunctionEmptyArrayUInt32 = FunctionEmptyArray<DataTypeUInt32>;
|
||||||
using FunctionEmptyArrayUInt64 = EmptyArray<DataTypeUInt64>;
|
using FunctionEmptyArrayUInt64 = FunctionEmptyArray<DataTypeUInt64>;
|
||||||
using FunctionEmptyArrayInt8 = EmptyArray<DataTypeInt8>;
|
using FunctionEmptyArrayInt8 = FunctionEmptyArray<DataTypeInt8>;
|
||||||
using FunctionEmptyArrayInt16 = EmptyArray<DataTypeInt16>;
|
using FunctionEmptyArrayInt16 = FunctionEmptyArray<DataTypeInt16>;
|
||||||
using FunctionEmptyArrayInt32 = EmptyArray<DataTypeInt32>;
|
using FunctionEmptyArrayInt32 = FunctionEmptyArray<DataTypeInt32>;
|
||||||
using FunctionEmptyArrayInt64 = EmptyArray<DataTypeInt64>;
|
using FunctionEmptyArrayInt64 = FunctionEmptyArray<DataTypeInt64>;
|
||||||
using FunctionEmptyArrayFloat32 = EmptyArray<DataTypeFloat32>;
|
using FunctionEmptyArrayFloat32 = FunctionEmptyArray<DataTypeFloat32>;
|
||||||
using FunctionEmptyArrayFloat64 = EmptyArray<DataTypeFloat64>;
|
using FunctionEmptyArrayFloat64 = FunctionEmptyArray<DataTypeFloat64>;
|
||||||
using FunctionEmptyArrayDate = EmptyArray<DataTypeDate>;
|
using FunctionEmptyArrayDate = FunctionEmptyArray<DataTypeDate>;
|
||||||
using FunctionEmptyArrayDateTime = EmptyArray<DataTypeDateTime>;
|
using FunctionEmptyArrayDateTime = FunctionEmptyArray<DataTypeDateTime>;
|
||||||
using FunctionEmptyArrayString = EmptyArray<DataTypeString>;
|
using FunctionEmptyArrayString = FunctionEmptyArray<DataTypeString>;
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -45,7 +45,10 @@ const auto ipv6_bytes_length = 16;
|
|||||||
class FunctionIPv6NumToString : public IFunction
|
class FunctionIPv6NumToString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
String getName() const { return "IPv6NumToString"; }
|
static constexpr auto name = "IPv6NumToString";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; }
|
||||||
|
|
||||||
|
String getName() const { return name; }
|
||||||
|
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
{
|
{
|
||||||
@ -247,7 +250,10 @@ public:
|
|||||||
class FunctionIPv6StringToNum : public IFunction
|
class FunctionIPv6StringToNum : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
String getName() const { return "IPv6StringToNum"; }
|
static constexpr auto name = "IPv6StringToNum";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIPv6StringToNum; }
|
||||||
|
|
||||||
|
String getName() const { return name; }
|
||||||
|
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
{
|
{
|
||||||
@ -462,10 +468,13 @@ public:
|
|||||||
class FunctionIPv4NumToString : public IFunction
|
class FunctionIPv4NumToString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "IPv4NumToString";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIPv4NumToString; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "IPv4NumToString";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -564,10 +573,13 @@ public:
|
|||||||
class FunctionIPv4StringToNum : public IFunction
|
class FunctionIPv4StringToNum : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "IPv4StringToNum";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIPv4StringToNum; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "IPv4StringToNum";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -652,10 +664,13 @@ public:
|
|||||||
class FunctionHex : public IFunction
|
class FunctionHex : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "hex";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionHex; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "hex";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -897,10 +912,13 @@ public:
|
|||||||
class FunctionUnhex : public IFunction
|
class FunctionUnhex : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "unhex";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionUnhex; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "unhex";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1002,10 +1020,13 @@ public:
|
|||||||
class FunctionBitmaskToArray : public IFunction
|
class FunctionBitmaskToArray : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "bitmaskToArray";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionBitmaskToArray; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "bitmaskToArray";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1111,10 +1132,13 @@ public:
|
|||||||
class FunctionToStringCutToZero : public IFunction
|
class FunctionToStringCutToZero : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "toStringCutToZero";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionToStringCutToZero; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "toStringCutToZero";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
@ -385,8 +385,11 @@ template <
|
|||||||
typename Name>
|
typename Name>
|
||||||
class FunctionComparison : public IFunction
|
class FunctionComparison : public IFunction
|
||||||
{
|
{
|
||||||
private:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionComparison; };
|
||||||
|
|
||||||
|
private:
|
||||||
template <typename T0, typename T1>
|
template <typename T0, typename T1>
|
||||||
bool executeNumRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
bool executeNumRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||||
{
|
{
|
||||||
@ -567,7 +570,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -615,12 +618,12 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameEquals { static const char * get() { return "equals"; } };
|
struct NameEquals { static constexpr auto name = "equals"; };
|
||||||
struct NameNotEquals { static const char * get() { return "notEquals"; } };
|
struct NameNotEquals { static constexpr auto name = "notEquals"; };
|
||||||
struct NameLess { static const char * get() { return "less"; } };
|
struct NameLess { static constexpr auto name = "less"; };
|
||||||
struct NameGreater { static const char * get() { return "greater"; } };
|
struct NameGreater { static constexpr auto name = "greater"; };
|
||||||
struct NameLessOrEquals { static const char * get() { return "lessOrEquals"; } };
|
struct NameLessOrEquals { static constexpr auto name = "lessOrEquals"; };
|
||||||
struct NameGreaterOrEquals { static const char * get() { return "greaterOrEquals"; } };
|
struct NameGreaterOrEquals { static constexpr auto name = "greaterOrEquals"; };
|
||||||
|
|
||||||
typedef FunctionComparison<EqualsOp, NameEquals> FunctionEquals;
|
typedef FunctionComparison<EqualsOp, NameEquals> FunctionEquals;
|
||||||
typedef FunctionComparison<NotEqualsOp, NameNotEquals> FunctionNotEquals;
|
typedef FunctionComparison<NotEqualsOp, NameNotEquals> FunctionNotEquals;
|
||||||
|
@ -296,6 +296,10 @@ struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
|
|||||||
|
|
||||||
class FunctionIf : public IFunction
|
class FunctionIf : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "if";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIf; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
template <typename T0, typename T1>
|
template <typename T0, typename T1>
|
||||||
bool checkRightType(const DataTypes & arguments, DataTypePtr & type_res) const
|
bool checkRightType(const DataTypes & arguments, DataTypePtr & type_res) const
|
||||||
@ -473,7 +477,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "if";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
@ -51,7 +51,7 @@ struct ConvertImpl
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -91,7 +91,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -128,7 +128,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -178,7 +178,7 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -240,7 +240,7 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -300,7 +300,7 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -356,7 +356,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -366,10 +366,13 @@ template <typename ToDataType, typename Name>
|
|||||||
class FunctionConvert : public IFunction
|
class FunctionConvert : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionConvert; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -414,10 +417,13 @@ public:
|
|||||||
class FunctionToFixedString : public IFunction
|
class FunctionToFixedString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "toFixedString";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionToFixedString; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "toFixedString";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Получить тип результата по типам аргументов и значениям константных аргументов.
|
/** Получить тип результата по типам аргументов и значениям константных аргументов.
|
||||||
@ -515,19 +521,19 @@ private:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameToUInt8 { static const char * get() { return "toUInt8"; } };
|
struct NameToUInt8 { static constexpr auto name = "toUInt8"; };
|
||||||
struct NameToUInt16 { static const char * get() { return "toUInt16"; } };
|
struct NameToUInt16 { static constexpr auto name = "toUInt16"; };
|
||||||
struct NameToUInt32 { static const char * get() { return "toUInt32"; } };
|
struct NameToUInt32 { static constexpr auto name = "toUInt32"; };
|
||||||
struct NameToUInt64 { static const char * get() { return "toUInt64"; } };
|
struct NameToUInt64 { static constexpr auto name = "toUInt64"; };
|
||||||
struct NameToInt8 { static const char * get() { return "toInt8"; } };
|
struct NameToInt8 { static constexpr auto name = "toInt8"; };
|
||||||
struct NameToInt16 { static const char * get() { return "toInt16"; } };
|
struct NameToInt16 { static constexpr auto name = "toInt16"; };
|
||||||
struct NameToInt32 { static const char * get() { return "toInt32"; } };
|
struct NameToInt32 { static constexpr auto name = "toInt32"; };
|
||||||
struct NameToInt64 { static const char * get() { return "toInt64"; } };
|
struct NameToInt64 { static constexpr auto name = "toInt64"; };
|
||||||
struct NameToFloat32 { static const char * get() { return "toFloat32"; } };
|
struct NameToFloat32 { static constexpr auto name = "toFloat32"; };
|
||||||
struct NameToFloat64 { static const char * get() { return "toFloat64"; } };
|
struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
|
||||||
struct NameToDate { static const char * get() { return "toDate"; } };
|
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||||
struct NameToDateTime { static const char * get() { return "toDateTime"; } };
|
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||||
struct NameToString { static const char * get() { return "toString"; } };
|
struct NameToString { static constexpr auto name = "toString"; };
|
||||||
|
|
||||||
typedef FunctionConvert<DataTypeUInt8, NameToUInt8> FunctionToUInt8;
|
typedef FunctionConvert<DataTypeUInt8, NameToUInt8> FunctionToUInt8;
|
||||||
typedef FunctionConvert<DataTypeUInt16, NameToUInt16> FunctionToUInt16;
|
typedef FunctionConvert<DataTypeUInt16, NameToUInt16> FunctionToUInt16;
|
||||||
|
@ -229,7 +229,7 @@ struct DateTimeTransformImpl
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -239,10 +239,13 @@ template <typename ToDataType, typename Transform, typename Name>
|
|||||||
class FunctionDateOrDateTimeToSomething : public IFunction
|
class FunctionDateOrDateTimeToSomething : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionDateOrDateTimeToSomething; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -276,10 +279,13 @@ public:
|
|||||||
class FunctionNow : public IFunction
|
class FunctionNow : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "now";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionNow; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "now";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -306,10 +312,13 @@ public:
|
|||||||
class FunctionToday : public IFunction
|
class FunctionToday : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "today";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionToday; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "today";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -336,10 +345,13 @@ public:
|
|||||||
class FunctionYesterday : public IFunction
|
class FunctionYesterday : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "yesterday";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionYesterday; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "yesterday";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -366,10 +378,13 @@ public:
|
|||||||
class FunctionTimeSlot : public IFunction
|
class FunctionTimeSlot : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "timeSlot";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionTimeSlot; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "timeSlot";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -498,10 +513,13 @@ struct TimeSlotsImpl
|
|||||||
class FunctionTimeSlots : public IFunction
|
class FunctionTimeSlots : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "timeSlots";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionTimeSlots; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "timeSlots";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -565,27 +583,27 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameToYear { static const char * get() { return "toYear"; } };
|
struct NameToYear { static constexpr auto name = "toYear"; };
|
||||||
struct NameToMonth { static const char * get() { return "toMonth"; } };
|
struct NameToMonth { static constexpr auto name = "toMonth"; };
|
||||||
struct NameToDayOfMonth { static const char * get() { return "toDayOfMonth"; } };
|
struct NameToDayOfMonth { static constexpr auto name = "toDayOfMonth"; };
|
||||||
struct NameToDayOfWeek { static const char * get() { return "toDayOfWeek"; } };
|
struct NameToDayOfWeek { static constexpr auto name = "toDayOfWeek"; };
|
||||||
struct NameToHour { static const char * get() { return "toHour"; } };
|
struct NameToHour { static constexpr auto name = "toHour"; };
|
||||||
struct NameToMinute { static const char * get() { return "toMinute"; } };
|
struct NameToMinute { static constexpr auto name = "toMinute"; };
|
||||||
struct NameToSecond { static const char * get() { return "toSecond"; } };
|
struct NameToSecond { static constexpr auto name = "toSecond"; };
|
||||||
struct NameToMonday { static const char * get() { return "toMonday"; } };
|
struct NameToMonday { static constexpr auto name = "toMonday"; };
|
||||||
struct NameToStartOfMonth { static const char * get() { return "toStartOfMonth"; } };
|
struct NameToStartOfMonth { static constexpr auto name = "toStartOfMonth"; };
|
||||||
struct NameToStartOfQuarter { static const char * get() { return "toStartOfQuarter"; } };
|
struct NameToStartOfQuarter { static constexpr auto name = "toStartOfQuarter"; };
|
||||||
struct NameToStartOfYear { static const char * get() { return "toStartOfYear"; } };
|
struct NameToStartOfYear { static constexpr auto name = "toStartOfYear"; };
|
||||||
struct NameToStartOfMinute { static const char * get() { return "toStartOfMinute"; } };
|
struct NameToStartOfMinute { static constexpr auto name = "toStartOfMinute"; };
|
||||||
struct NameToStartOfHour { static const char * get() { return "toStartOfHour"; } };
|
struct NameToStartOfHour { static constexpr auto name = "toStartOfHour"; };
|
||||||
struct NameToTime { static const char * get() { return "toTime"; } };
|
struct NameToTime { static constexpr auto name = "toTime"; };
|
||||||
struct NameToRelativeYearNum { static const char * get() { return "toRelativeYearNum"; } };
|
struct NameToRelativeYearNum { static constexpr auto name = "toRelativeYearNum"; };
|
||||||
struct NameToRelativeMonthNum { static const char * get() { return "toRelativeMonthNum"; } };
|
struct NameToRelativeMonthNum { static constexpr auto name = "toRelativeMonthNum"; };
|
||||||
struct NameToRelativeWeekNum { static const char * get() { return "toRelativeWeekNum"; } };
|
struct NameToRelativeWeekNum { static constexpr auto name = "toRelativeWeekNum"; };
|
||||||
struct NameToRelativeDayNum { static const char * get() { return "toRelativeDayNum"; } };
|
struct NameToRelativeDayNum { static constexpr auto name = "toRelativeDayNum"; };
|
||||||
struct NameToRelativeHourNum { static const char * get() { return "toRelativeHourNum"; } };
|
struct NameToRelativeHourNum { static constexpr auto name = "toRelativeHourNum"; };
|
||||||
struct NameToRelativeMinuteNum { static const char * get() { return "toRelativeMinuteNum"; } };
|
struct NameToRelativeMinuteNum { static constexpr auto name = "toRelativeMinuteNum"; };
|
||||||
struct NameToRelativeSecondNum { static const char * get() { return "toRelativeSecondNum"; } };
|
struct NameToRelativeSecondNum { static constexpr auto name = "toRelativeSecondNum"; };
|
||||||
|
|
||||||
|
|
||||||
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl, NameToYear> FunctionToYear;
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl, NameToYear> FunctionToYear;
|
||||||
|
@ -161,6 +161,10 @@ struct IdentityDictionaryGetter
|
|||||||
template <typename T, typename Transform, typename DictGetter, typename Name>
|
template <typename T, typename Transform, typename DictGetter, typename Name>
|
||||||
class FunctionTransformWithDictionary : public IFunction
|
class FunctionTransformWithDictionary : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
using base_type = FunctionTransformWithDictionary;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||||
|
|
||||||
@ -175,7 +179,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -211,7 +215,7 @@ public:
|
|||||||
|
|
||||||
if (!key_col)
|
if (!key_col)
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
+ " of second ('point of view') argument of function " + Name::get()
|
+ " of second ('point of view') argument of function " + name
|
||||||
+ ". Must be constant string.",
|
+ ". Must be constant string.",
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
@ -239,7 +243,7 @@ public:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -249,6 +253,10 @@ public:
|
|||||||
template <typename T, typename Transform, typename DictGetter, typename Name>
|
template <typename T, typename Transform, typename DictGetter, typename Name>
|
||||||
class FunctionIsInWithDictionary : public IFunction
|
class FunctionIsInWithDictionary : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
using base_type = FunctionIsInWithDictionary;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||||
|
|
||||||
@ -263,7 +271,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -304,7 +312,7 @@ public:
|
|||||||
|
|
||||||
if (!key_col)
|
if (!key_col)
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||||
+ " of third ('point of view') argument of function " + Name::get()
|
+ " of third ('point of view') argument of function " + name
|
||||||
+ ". Must be constant string.",
|
+ ". Must be constant string.",
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
@ -368,7 +376,7 @@ public:
|
|||||||
else
|
else
|
||||||
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " and " + block.getByPosition(arguments[1]).column->getName()
|
+ " and " + block.getByPosition(arguments[1]).column->getName()
|
||||||
+ " of arguments of function " + Name::get(),
|
+ " of arguments of function " + name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -378,6 +386,10 @@ public:
|
|||||||
template <typename T, typename Transform, typename DictGetter, typename Name>
|
template <typename T, typename Transform, typename DictGetter, typename Name>
|
||||||
class FunctionHierarchyWithDictionary : public IFunction
|
class FunctionHierarchyWithDictionary : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
using base_type = FunctionHierarchyWithDictionary;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||||
|
|
||||||
@ -392,7 +404,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -428,7 +440,7 @@ public:
|
|||||||
|
|
||||||
if (!key_col)
|
if (!key_col)
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
+ " of second ('point of view') argument of function " + Name::get()
|
+ " of second ('point of view') argument of function " + name
|
||||||
+ ". Must be constant string.",
|
+ ". Must be constant string.",
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
@ -477,88 +489,197 @@ public:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameRegionToCity { static const char * get() { return "regionToCity"; } };
|
struct NameRegionToCity { static constexpr auto name = "regionToCity"; };
|
||||||
struct NameRegionToArea { static const char * get() { return "regionToArea"; } };
|
struct NameRegionToArea { static constexpr auto name = "regionToArea"; };
|
||||||
struct NameRegionToDistrict { static const char * get() { return "regionToDistrict"; } };
|
struct NameRegionToDistrict { static constexpr auto name = "regionToDistrict"; };
|
||||||
struct NameRegionToCountry { static const char * get() { return "regionToCountry"; } };
|
struct NameRegionToCountry { static constexpr auto name = "regionToCountry"; };
|
||||||
struct NameRegionToContinent{ static const char * get() { return "regionToContient"; } };
|
struct NameRegionToContinent { static constexpr auto name = "regionToContient"; };
|
||||||
struct NameOSToRoot { static const char * get() { return "OSToRoot"; } };
|
struct NameOSToRoot { static constexpr auto name = "OSToRoot"; };
|
||||||
struct NameSEToRoot { static const char * get() { return "SEToRoot"; } };
|
struct NameSEToRoot { static constexpr auto name = "SEToRoot"; };
|
||||||
struct NameCategoryToRoot { static const char * get() { return "categoryToRoot"; } };
|
struct NameCategoryToRoot { static constexpr auto name = "categoryToRoot"; };
|
||||||
struct NameCategoryToSecondLevel { static const char * get() { return "categoryToSecondLevel"; } };
|
struct NameCategoryToSecondLevel { static constexpr auto name = "categoryToSecondLevel"; };
|
||||||
|
|
||||||
struct NameRegionIn { static const char * get() { return "regionIn"; } };
|
struct NameRegionIn { static constexpr auto name = "regionIn"; };
|
||||||
struct NameOSIn { static const char * get() { return "OSIn"; } };
|
struct NameOSIn { static constexpr auto name = "OSIn"; };
|
||||||
struct NameSEIn { static const char * get() { return "SEIn"; } };
|
struct NameSEIn { static constexpr auto name = "SEIn"; };
|
||||||
struct NameCategoryIn { static const char * get() { return "categoryIn"; } };
|
struct NameCategoryIn { static constexpr auto name = "categoryIn"; };
|
||||||
|
|
||||||
struct NameRegionHierarchy { static const char * get() { return "regionHierarchy"; } };
|
struct NameRegionHierarchy { static constexpr auto name = "regionHierarchy"; };
|
||||||
struct NameOSHierarchy { static const char * get() { return "OSHierarchy"; } };
|
struct NameOSHierarchy { static constexpr auto name = "OSHierarchy"; };
|
||||||
struct NameSEHierarchy { static const char * get() { return "SEHierarchy"; } };
|
struct NameSEHierarchy { static constexpr auto name = "SEHierarchy"; };
|
||||||
struct NameCategoryHierarchy{ static const char * get() { return "categoryHierarchy"; } };
|
struct NameCategoryHierarchy { static constexpr auto name = "categoryHierarchy"; };
|
||||||
|
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionRegionToCity :
|
||||||
<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity> FunctionRegionToCity;
|
public FunctionTransformWithDictionary<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionRegionToArea :
|
||||||
<UInt32, RegionToAreaImpl, RegionsHierarchyGetter, NameRegionToArea> FunctionRegionToArea;
|
public FunctionTransformWithDictionary<UInt32, RegionToAreaImpl, RegionsHierarchyGetter, NameRegionToArea>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionRegionToDistrict :
|
||||||
<UInt32, RegionToDistrictImpl, RegionsHierarchyGetter, NameRegionToDistrict> FunctionRegionToDistrict;
|
public FunctionTransformWithDictionary<UInt32, RegionToDistrictImpl, RegionsHierarchyGetter, NameRegionToDistrict>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionRegionToCountry :
|
||||||
<UInt32, RegionToCountryImpl, RegionsHierarchyGetter, NameRegionToCountry> FunctionRegionToCountry;
|
public FunctionTransformWithDictionary<UInt32, RegionToCountryImpl, RegionsHierarchyGetter, NameRegionToCountry>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionRegionToContinent :
|
||||||
<UInt32, RegionToContinentImpl, RegionsHierarchyGetter, NameRegionToContinent> FunctionRegionToContinent;
|
public FunctionTransformWithDictionary<UInt32, RegionToContinentImpl, RegionsHierarchyGetter, NameRegionToContinent>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionOSToRoot :
|
||||||
<UInt8, OSToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSToRoot> FunctionOSToRoot;
|
public FunctionTransformWithDictionary<UInt8, OSToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSToRoot>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getTechDataHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionSEToRoot :
|
||||||
<UInt8, SEToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEToRoot> FunctionSEToRoot;
|
public FunctionTransformWithDictionary<UInt8, SEToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEToRoot>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getTechDataHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionCategoryToRoot :
|
||||||
<UInt16, CategoryToRootImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToRoot> FunctionCategoryToRoot;
|
public FunctionTransformWithDictionary<UInt16, CategoryToRootImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToRoot>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionTransformWithDictionary
|
struct FunctionCategoryToSecondLevel :
|
||||||
<UInt16, CategoryToSecondLevelImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToSecondLevel> FunctionCategoryToSecondLevel;
|
public FunctionTransformWithDictionary<UInt16, CategoryToSecondLevelImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryToSecondLevel>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionIsInWithDictionary
|
struct FunctionRegionIn :
|
||||||
<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn> FunctionRegionIn;
|
public FunctionIsInWithDictionary<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionIsInWithDictionary
|
struct FunctionOSIn :
|
||||||
<UInt8, OSInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSIn> FunctionOSIn;
|
public FunctionIsInWithDictionary<UInt8, OSInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSIn>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getTechDataHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionIsInWithDictionary
|
struct FunctionSEIn :
|
||||||
<UInt8, SEInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEIn> FunctionSEIn;
|
public FunctionIsInWithDictionary<UInt8, SEInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEIn>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getTechDataHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionIsInWithDictionary
|
struct FunctionCategoryIn :
|
||||||
<UInt16, CategoryInImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryIn> FunctionCategoryIn;
|
public FunctionIsInWithDictionary<UInt16, CategoryInImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryIn>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionHierarchyWithDictionary
|
struct FunctionRegionHierarchy :
|
||||||
<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy> FunctionRegionHierarchy;
|
public FunctionHierarchyWithDictionary<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getRegionsHierarchies()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionHierarchyWithDictionary
|
struct FunctionOSHierarchy :
|
||||||
<UInt8, OSHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSHierarchy> FunctionOSHierarchy;
|
public FunctionHierarchyWithDictionary<UInt8, OSHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSHierarchy>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getTechDataHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionHierarchyWithDictionary
|
struct FunctionSEHierarchy :
|
||||||
<UInt8, SEHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEHierarchy> FunctionSEHierarchy;
|
public FunctionHierarchyWithDictionary<UInt8, SEHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEHierarchy>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getTechDataHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
typedef FunctionHierarchyWithDictionary
|
struct FunctionCategoryHierarchy :
|
||||||
<UInt16, CategoryHierarchyImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryHierarchy> FunctionCategoryHierarchy;
|
public FunctionHierarchyWithDictionary<UInt16, CategoryHierarchyImpl, IdentityDictionaryGetter<CategoriesHierarchy>, NameCategoryHierarchy>
|
||||||
|
{
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new base_type{context.getDictionaries().getCategoriesHierarchy()};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
/// Преобразует числовой идентификатор региона в имя на заданном языке, используя словарь.
|
/// Преобразует числовой идентификатор региона в имя на заданном языке, используя словарь.
|
||||||
class FunctionRegionToName : public IFunction
|
class FunctionRegionToName : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "regionToName";
|
||||||
|
static IFunction * create(const Context & context)
|
||||||
|
{
|
||||||
|
return new FunctionRegionToName(context.getDictionaries().getRegionsNames());
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const SharedPtr<RegionsNames> owned_dict;
|
const SharedPtr<RegionsNames> owned_dict;
|
||||||
|
|
||||||
@ -573,7 +694,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "regionToName";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
@ -18,10 +18,13 @@ namespace DB
|
|||||||
class FunctionBitmaskToList : public IFunction
|
class FunctionBitmaskToList : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "bitmaskToList";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionBitmaskToList; }
|
||||||
|
|
||||||
/// Получить основное имя функции.
|
/// Получить основное имя функции.
|
||||||
virtual String getName() const
|
virtual String getName() const
|
||||||
{
|
{
|
||||||
return "bitmaskToList";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
@ -166,10 +166,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionStringHash64 : public IFunction
|
class FunctionStringHash64 : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringHash64; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -214,7 +217,7 @@ public:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -224,10 +227,13 @@ template <typename Impl>
|
|||||||
class FunctionStringHashFixedString : public IFunction
|
class FunctionStringHashFixedString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Impl::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringHashFixedString; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Impl::name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -289,6 +295,10 @@ public:
|
|||||||
template <typename Impl, typename Name>
|
template <typename Impl, typename Name>
|
||||||
class FunctionIntHash : public IFunction
|
class FunctionIntHash : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIntHash; };
|
||||||
|
|
||||||
private:
|
private:
|
||||||
typedef typename Impl::ReturnType ToType;
|
typedef typename Impl::ReturnType ToType;
|
||||||
|
|
||||||
@ -314,7 +324,7 @@ private:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
+ " of first argument of function " + Name::get(),
|
+ " of first argument of function " + Name::name,
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -322,7 +332,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -387,6 +397,10 @@ UInt64 toInteger<Float64>(Float64 x)
|
|||||||
|
|
||||||
class FunctionCityHash64 : public IFunction
|
class FunctionCityHash64 : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "cityHash64";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionCityHash64; };
|
||||||
|
|
||||||
private:
|
private:
|
||||||
template <typename FromType, bool first>
|
template <typename FromType, bool first>
|
||||||
void executeIntType(const IColumn * column, ColumnUInt64::Container_t & vec_to)
|
void executeIntType(const IColumn * column, ColumnUInt64::Container_t & vec_to)
|
||||||
@ -550,7 +564,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "cityHash64";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -589,11 +603,11 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameHalfMD5 { static const char * get() { return "halfMD5"; } };
|
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
|
||||||
struct NameSipHash64 { static const char * get() { return "sipHash64"; } };
|
struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
|
||||||
struct NameCityHash64 { static const char * get() { return "cityHash64"; } };
|
struct NameCityHash64 { static constexpr auto name = "cityHash64"; };
|
||||||
struct NameIntHash32 { static const char * get() { return "intHash32"; } };
|
struct NameIntHash32 { static constexpr auto name = "intHash32"; };
|
||||||
struct NameIntHash64 { static const char * get() { return "intHash64"; } };
|
struct NameIntHash64 { static constexpr auto name = "intHash64"; };
|
||||||
|
|
||||||
typedef FunctionStringHash64<HalfMD5Impl, NameHalfMD5> FunctionHalfMD5;
|
typedef FunctionStringHash64<HalfMD5Impl, NameHalfMD5> FunctionHalfMD5;
|
||||||
typedef FunctionStringHash64<SipHash64Impl, NameSipHash64> FunctionSipHash64;
|
typedef FunctionStringHash64<SipHash64Impl, NameSipHash64> FunctionSipHash64;
|
||||||
|
@ -381,10 +381,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionArrayMapped : public IFunction
|
class FunctionArrayMapped : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionArrayMapped; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Вызывается, если хоть один агрумент функции - лямбда-выражение.
|
/// Вызывается, если хоть один агрумент функции - лямбда-выражение.
|
||||||
@ -591,14 +594,14 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameArrayMap { static const char * get() { return "arrayMap"; } };
|
struct NameArrayMap { static constexpr auto name = "arrayMap"; };
|
||||||
struct NameArrayFilter { static const char * get() { return "arrayFilter"; } };
|
struct NameArrayFilter { static constexpr auto name = "arrayFilter"; };
|
||||||
struct NameArrayCount { static const char * get() { return "arrayCount"; } };
|
struct NameArrayCount { static constexpr auto name = "arrayCount"; };
|
||||||
struct NameArrayExists { static const char * get() { return "arrayExists"; } };
|
struct NameArrayExists { static constexpr auto name = "arrayExists"; };
|
||||||
struct NameArrayAll { static const char * get() { return "arrayAll"; } };
|
struct NameArrayAll { static constexpr auto name = "arrayAll"; };
|
||||||
struct NameArraySum { static const char * get() { return "arraySum"; } };
|
struct NameArraySum { static constexpr auto name = "arraySum"; };
|
||||||
struct NameArrayFirst { static const char * get() { return "arrayFirst"; } };
|
struct NameArrayFirst { static constexpr auto name = "arrayFirst"; };
|
||||||
struct NameArrayFirstIndex { static const char * get() { return "arrayFirstIndex"; } };
|
struct NameArrayFirstIndex { static constexpr auto name = "arrayFirstIndex"; };
|
||||||
|
|
||||||
typedef FunctionArrayMapped<ArrayMapImpl, NameArrayMap> FunctionArrayMap;
|
typedef FunctionArrayMapped<ArrayMapImpl, NameArrayMap> FunctionArrayMap;
|
||||||
typedef FunctionArrayMapped<ArrayFilterImpl, NameArrayFilter> FunctionArrayFilter;
|
typedef FunctionArrayMapped<ArrayFilterImpl, NameArrayFilter> FunctionArrayFilter;
|
||||||
|
@ -152,6 +152,10 @@ struct AssociativeOperationImpl<Op, 1>
|
|||||||
template <template <typename> class Impl, typename Name>
|
template <template <typename> class Impl, typename Name>
|
||||||
class FunctionAnyArityLogical : public IFunction
|
class FunctionAnyArityLogical : public IFunction
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionAnyArityLogical; };
|
||||||
|
|
||||||
private:
|
private:
|
||||||
bool extractConstColumns(ColumnPlainPtrs & in, UInt8 & res)
|
bool extractConstColumns(ColumnPlainPtrs & in, UInt8 & res)
|
||||||
{
|
{
|
||||||
@ -240,7 +244,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -357,8 +361,11 @@ public:
|
|||||||
template <template <typename> class Impl, typename Name>
|
template <template <typename> class Impl, typename Name>
|
||||||
class FunctionUnaryLogical : public IFunction
|
class FunctionUnaryLogical : public IFunction
|
||||||
{
|
{
|
||||||
private:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionUnaryLogical; };
|
||||||
|
|
||||||
|
private:
|
||||||
template <typename T>
|
template <typename T>
|
||||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
@ -391,7 +398,7 @@ public:
|
|||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -431,10 +438,10 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameAnd { static const char * get() { return "and"; } };
|
struct NameAnd { static constexpr auto name = "and"; };
|
||||||
struct NameOr { static const char * get() { return "or"; } };
|
struct NameOr { static constexpr auto name = "or"; };
|
||||||
struct NameXor { static const char * get() { return "xor"; } };
|
struct NameXor { static constexpr auto name = "xor"; };
|
||||||
struct NameNot { static const char * get() { return "not"; } };
|
struct NameNot { static constexpr auto name = "not"; };
|
||||||
|
|
||||||
typedef FunctionAnyArityLogical <AndImpl, NameAnd> FunctionAnd;
|
typedef FunctionAnyArityLogical <AndImpl, NameAnd> FunctionAnd;
|
||||||
typedef FunctionAnyArityLogical <OrImpl, NameOr> FunctionOr;
|
typedef FunctionAnyArityLogical <OrImpl, NameOr> FunctionOr;
|
||||||
|
@ -76,13 +76,16 @@ static inline void stringWidthConstant(const String & data, UInt64 & res)
|
|||||||
|
|
||||||
class FunctionCurrentDatabase : public IFunction
|
class FunctionCurrentDatabase : public IFunction
|
||||||
{
|
{
|
||||||
const String name;
|
const String db_name;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
explicit FunctionCurrentDatabase(const String & name) : name(name) {}
|
static constexpr auto name = "currentDatabase";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionCurrentDatabase{context.getCurrentDatabase()}; }
|
||||||
|
|
||||||
|
explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} {}
|
||||||
|
|
||||||
String getName() const {
|
String getName() const {
|
||||||
return "currentDatabase";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
@ -98,7 +101,7 @@ public:
|
|||||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||||
{
|
{
|
||||||
block.getByPosition(result).column = new ColumnConstString{
|
block.getByPosition(result).column = new ColumnConstString{
|
||||||
block.rowsInFirstColumn(), name
|
block.rowsInFirstColumn(), db_name
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -107,10 +110,13 @@ public:
|
|||||||
class FunctionHostName : public IFunction
|
class FunctionHostName : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "hostName";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionHostName; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "hostName";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -124,7 +130,8 @@ public:
|
|||||||
return new DataTypeString;
|
return new DataTypeString;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Выполнить функцию над блоком.
|
/** Выполнить функцию над блоком. convertToFullColumn вызывается для того, чтобы в случае
|
||||||
|
* распределенного выполнения запроса каждый сервер возвращал свое имя хоста. */
|
||||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||||
{
|
{
|
||||||
block.getByPosition(result).column = ColumnConstString(
|
block.getByPosition(result).column = ColumnConstString(
|
||||||
@ -136,10 +143,13 @@ public:
|
|||||||
class FunctionVisibleWidth : public IFunction
|
class FunctionVisibleWidth : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "visibleWidth";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionVisibleWidth; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "visibleWidth";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -161,10 +171,13 @@ public:
|
|||||||
class FunctionToTypeName : public IFunction
|
class FunctionToTypeName : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "toTypeName";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionToTypeName; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "toTypeName";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -189,10 +202,13 @@ public:
|
|||||||
class FunctionBlockSize : public IFunction
|
class FunctionBlockSize : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "blockSize";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionBlockSize; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "blockSize";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -218,10 +234,13 @@ public:
|
|||||||
class FunctionSleep : public IFunction
|
class FunctionSleep : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "sleep";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionSleep; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "sleep";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -284,10 +303,13 @@ public:
|
|||||||
class FunctionMaterialize : public IFunction
|
class FunctionMaterialize : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "materialize";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionMaterialize; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "materialize";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -312,23 +334,23 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
template <bool negative, bool global> struct FunctionInName;
|
||||||
|
template <> struct FunctionInName<false, false> { static constexpr auto name = "in"; };
|
||||||
|
template <> struct FunctionInName<false, true> { static constexpr auto name = "globalIn"; };
|
||||||
|
template <> struct FunctionInName<true, false> { static constexpr auto name = "notIn"; };
|
||||||
|
template <> struct FunctionInName<true, true> { static constexpr auto name = "globalNotIn"; };
|
||||||
|
|
||||||
|
template <bool negative, bool global>
|
||||||
class FunctionIn : public IFunction
|
class FunctionIn : public IFunction
|
||||||
{
|
{
|
||||||
private:
|
|
||||||
bool negative;
|
|
||||||
bool global;
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
FunctionIn(bool negative_ = false, bool global_ = false) : negative(negative_), global(global_) {}
|
static constexpr auto name = FunctionInName<negative, global>::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIn; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
if (global)
|
return name;
|
||||||
return negative ? "globalNotIn" : "globalIn";
|
|
||||||
else
|
|
||||||
return negative ? "notIn" : "in";
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -376,10 +398,13 @@ public:
|
|||||||
class FunctionTuple : public IFunction
|
class FunctionTuple : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "tuple";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionTuple; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "tuple";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -407,10 +432,13 @@ public:
|
|||||||
class FunctionTupleElement : public IFunction
|
class FunctionTupleElement : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "tupleElement";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionTupleElement; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "tupleElement";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||||
@ -471,10 +499,13 @@ public:
|
|||||||
class FunctionIgnore : public IFunction
|
class FunctionIgnore : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "ignore";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIgnore; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "ignore";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -494,10 +525,14 @@ public:
|
|||||||
class FunctionArrayJoin : public IFunction
|
class FunctionArrayJoin : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "arrayJoin";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionArrayJoin; }
|
||||||
|
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "arrayJoin";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -529,10 +564,14 @@ public:
|
|||||||
*/
|
*/
|
||||||
class FunctionReplicate : public IFunction
|
class FunctionReplicate : public IFunction
|
||||||
{
|
{
|
||||||
|
static constexpr auto name = "replicate";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionReplicate; }
|
||||||
|
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "replicate";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -575,10 +614,13 @@ class FunctionReplicate : public IFunction
|
|||||||
class FunctionBar : public IFunction
|
class FunctionBar : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = "bar";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionBar; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return "bar";
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
@ -147,10 +147,13 @@ private:
|
|||||||
typedef typename Impl::ReturnType ToType;
|
typedef typename Impl::ReturnType ToType;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionRandom; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -179,8 +182,8 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameRand { static const char * get() { return "rand"; } };
|
struct NameRand { static constexpr auto name = "rand"; };
|
||||||
struct NameRand64 { static const char * get() { return "rand64"; } };
|
struct NameRand64 { static constexpr auto name = "rand64"; };
|
||||||
|
|
||||||
typedef FunctionRandom<RandImpl, NameRand> FunctionRand;
|
typedef FunctionRandom<RandImpl, NameRand> FunctionRand;
|
||||||
typedef FunctionRandom<Rand64Impl, NameRand64> FunctionRand64;
|
typedef FunctionRandom<Rand64Impl, NameRand64> FunctionRand64;
|
||||||
|
@ -23,10 +23,13 @@ template<typename Name>
|
|||||||
class FunctionReinterpretAsStringImpl : public IFunction
|
class FunctionReinterpretAsStringImpl : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionReinterpretAsStringImpl; };
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -116,12 +119,15 @@ template<typename ToDataType, typename Name>
|
|||||||
class FunctionReinterpretStringAs : public IFunction
|
class FunctionReinterpretStringAs : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionReinterpretStringAs; };
|
||||||
|
|
||||||
typedef typename ToDataType::FieldType ToFieldType;
|
typedef typename ToDataType::FieldType ToFieldType;
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -202,19 +208,19 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameReinterpretAsUInt8 { static const char * get() { return "reinterpretAsUInt8"; } };
|
struct NameReinterpretAsUInt8 { static constexpr auto name = "reinterpretAsUInt8"; };
|
||||||
struct NameReinterpretAsUInt16 { static const char * get() { return "reinterpretAsUInt16"; } };
|
struct NameReinterpretAsUInt16 { static constexpr auto name = "reinterpretAsUInt16"; };
|
||||||
struct NameReinterpretAsUInt32 { static const char * get() { return "reinterpretAsUInt32"; } };
|
struct NameReinterpretAsUInt32 { static constexpr auto name = "reinterpretAsUInt32"; };
|
||||||
struct NameReinterpretAsUInt64 { static const char * get() { return "reinterpretAsUInt64"; } };
|
struct NameReinterpretAsUInt64 { static constexpr auto name = "reinterpretAsUInt64"; };
|
||||||
struct NameReinterpretAsInt8 { static const char * get() { return "reinterpretAsInt8"; } };
|
struct NameReinterpretAsInt8 { static constexpr auto name = "reinterpretAsInt8"; };
|
||||||
struct NameReinterpretAsInt16 { static const char * get() { return "reinterpretAsInt16"; } };
|
struct NameReinterpretAsInt16 { static constexpr auto name = "reinterpretAsInt16"; };
|
||||||
struct NameReinterpretAsInt32 { static const char * get() { return "reinterpretAsInt32"; } };
|
struct NameReinterpretAsInt32 { static constexpr auto name = "reinterpretAsInt32"; };
|
||||||
struct NameReinterpretAsInt64 { static const char * get() { return "reinterpretAsInt64"; } };
|
struct NameReinterpretAsInt64 { static constexpr auto name = "reinterpretAsInt64"; };
|
||||||
struct NameReinterpretAsFloat32 { static const char * get() { return "reinterpretAsFloat32"; } };
|
struct NameReinterpretAsFloat32 { static constexpr auto name = "reinterpretAsFloat32"; };
|
||||||
struct NameReinterpretAsFloat64 { static const char * get() { return "reinterpretAsFloat64"; } };
|
struct NameReinterpretAsFloat64 { static constexpr auto name = "reinterpretAsFloat64"; };
|
||||||
struct NameReinterpretAsDate { static const char * get() { return "reinterpretAsDate"; } };
|
struct NameReinterpretAsDate { static constexpr auto name = "reinterpretAsDate"; };
|
||||||
struct NameReinterpretAsDateTime { static const char * get() { return "reinterpretAsDateTime"; } };
|
struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; };
|
||||||
struct NameReinterpretAsString { static const char * get() { return "reinterpretAsString"; } };
|
struct NameReinterpretAsString { static constexpr auto name = "reinterpretAsString"; };
|
||||||
|
|
||||||
typedef FunctionReinterpretStringAs<DataTypeUInt8, NameReinterpretAsUInt8> FunctionReinterpretAsUInt8;
|
typedef FunctionReinterpretStringAs<DataTypeUInt8, NameReinterpretAsUInt8> FunctionReinterpretAsUInt8;
|
||||||
typedef FunctionReinterpretStringAs<DataTypeUInt16, NameReinterpretAsUInt16> FunctionReinterpretAsUInt16;
|
typedef FunctionReinterpretStringAs<DataTypeUInt16, NameReinterpretAsUInt16> FunctionReinterpretAsUInt16;
|
||||||
|
@ -88,9 +88,9 @@ namespace DB
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameRoundToExp2 { static const char * get() { return "roundToExp2"; } };
|
struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; };
|
||||||
struct NameRoundDuration { static const char * get() { return "roundDuration"; } };
|
struct NameRoundDuration { static constexpr auto name = "roundDuration"; };
|
||||||
struct NameRoundAge { static const char * get() { return "roundAge"; } };
|
struct NameRoundAge { static constexpr auto name = "roundAge"; };
|
||||||
|
|
||||||
typedef FunctionUnaryArithmetic<RoundToExp2Impl, NameRoundToExp2> FunctionRoundToExp2;
|
typedef FunctionUnaryArithmetic<RoundToExp2Impl, NameRoundToExp2> FunctionRoundToExp2;
|
||||||
typedef FunctionUnaryArithmetic<RoundDurationImpl, NameRoundDuration> FunctionRoundDuration;
|
typedef FunctionUnaryArithmetic<RoundDurationImpl, NameRoundDuration> FunctionRoundDuration;
|
||||||
|
@ -809,10 +809,13 @@ template <typename Impl, typename Name, typename ResultType>
|
|||||||
class FunctionStringOrArrayToT : public IFunction
|
class FunctionStringOrArrayToT : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringOrArrayToT; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -902,10 +905,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionStringToString : public IFunction
|
class FunctionStringToString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringToString; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -960,10 +966,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionStringStringToString : public IFunction
|
class FunctionStringStringToString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringStringToString; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1212,10 +1221,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionStringNumNumToString : public IFunction
|
class FunctionStringNumNumToString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringNumNumToString; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1287,19 +1299,19 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameEmpty { static const char * get() { return "empty"; } };
|
struct NameEmpty { static constexpr auto name = "empty"; };
|
||||||
struct NameNotEmpty { static const char * get() { return "notEmpty"; } };
|
struct NameNotEmpty { static constexpr auto name = "notEmpty"; };
|
||||||
struct NameLength { static const char * get() { return "length"; } };
|
struct NameLength { static constexpr auto name = "length"; };
|
||||||
struct NameLengthUTF8 { static const char * get() { return "lengthUTF8"; } };
|
struct NameLengthUTF8 { static constexpr auto name = "lengthUTF8"; };
|
||||||
struct NameLower { static const char * get() { return "lower"; } };
|
struct NameLower { static constexpr auto name = "lower"; };
|
||||||
struct NameUpper { static const char * get() { return "upper"; } };
|
struct NameUpper { static constexpr auto name = "upper"; };
|
||||||
struct NameLowerUTF8 { static const char * get() { return "lowerUTF8"; } };
|
struct NameLowerUTF8 { static constexpr auto name = "lowerUTF8"; };
|
||||||
struct NameUpperUTF8 { static const char * get() { return "upperUTF8"; } };
|
struct NameUpperUTF8 { static constexpr auto name = "upperUTF8"; };
|
||||||
struct NameReverse { static const char * get() { return "reverse"; } };
|
struct NameReverse { static constexpr auto name = "reverse"; };
|
||||||
struct NameReverseUTF8 { static const char * get() { return "reverseUTF8"; } };
|
struct NameReverseUTF8 { static constexpr auto name = "reverseUTF8"; };
|
||||||
struct NameConcat { static const char * get() { return "concat"; } };
|
struct NameConcat { static constexpr auto name = "concat"; };
|
||||||
struct NameSubstring { static const char * get() { return "substring"; } };
|
struct NameSubstring { static constexpr auto name = "substring"; };
|
||||||
struct NameSubstringUTF8 { static const char * get() { return "substringUTF8"; } };
|
struct NameSubstringUTF8 { static constexpr auto name = "substringUTF8"; };
|
||||||
|
|
||||||
typedef FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8> FunctionEmpty;
|
typedef FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8> FunctionEmpty;
|
||||||
typedef FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8> FunctionNotEmpty;
|
typedef FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8> FunctionNotEmpty;
|
||||||
|
@ -45,7 +45,8 @@ private:
|
|||||||
|
|
||||||
public:
|
public:
|
||||||
/// Получить имя фукнции.
|
/// Получить имя фукнции.
|
||||||
static String getName() { return "alphaTokens"; }
|
static constexpr auto name = "alphaTokens";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
/// Проверить типы агрументов функции.
|
/// Проверить типы агрументов функции.
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
@ -107,7 +108,8 @@ private:
|
|||||||
char sep;
|
char sep;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static String getName() { return "splitByChar"; }
|
static constexpr auto name = "splitByChar";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
{
|
{
|
||||||
@ -183,7 +185,8 @@ private:
|
|||||||
String sep;
|
String sep;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static String getName() { return "splitByString"; }
|
static constexpr auto name = "splitByString";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
{
|
{
|
||||||
@ -246,7 +249,8 @@ private:
|
|||||||
Pos end;
|
Pos end;
|
||||||
public:
|
public:
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
static String getName() { return "extractAll"; }
|
static constexpr auto name = "extractAll";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
/// Проверить типы агрументов функции.
|
/// Проверить типы агрументов функции.
|
||||||
static void checkArguments( const DataTypes & arguments )
|
static void checkArguments( const DataTypes & arguments )
|
||||||
@ -305,10 +309,13 @@ template <typename Generator>
|
|||||||
class FunctionTokens : public IFunction
|
class FunctionTokens : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Generator::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionTokens; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Generator::getName();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
|
@ -870,10 +870,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionStringReplace : public IFunction
|
class FunctionStringReplace : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionStringReplace; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -954,10 +957,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionsStringSearch : public IFunction
|
class FunctionsStringSearch : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionsStringSearch; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1020,10 +1026,13 @@ template <typename Impl, typename Name>
|
|||||||
class FunctionsStringSearchToString : public IFunction
|
class FunctionsStringSearchToString : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionsStringSearchToString; }
|
||||||
|
|
||||||
/// Получить имя функции.
|
/// Получить имя функции.
|
||||||
String getName() const
|
String getName() const
|
||||||
{
|
{
|
||||||
return Name::get();
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||||
@ -1091,16 +1100,16 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NamePosition { static const char * get() { return "position"; } };
|
struct NamePosition { static constexpr auto name = "position"; };
|
||||||
struct NamePositionUTF8 { static const char * get() { return "positionUTF8"; } };
|
struct NamePositionUTF8 { static constexpr auto name = "positionUTF8"; };
|
||||||
struct NameMatch { static const char * get() { return "match"; } };
|
struct NameMatch { static constexpr auto name = "match"; };
|
||||||
struct NameLike { static const char * get() { return "like"; } };
|
struct NameLike { static constexpr auto name = "like"; };
|
||||||
struct NameNotLike { static const char * get() { return "notLike"; } };
|
struct NameNotLike { static constexpr auto name = "notLike"; };
|
||||||
struct NameExtract { static const char * get() { return "extract"; } };
|
struct NameExtract { static constexpr auto name = "extract"; };
|
||||||
struct NameReplaceOne { static const char * get() { return "replaceOne"; } };
|
struct NameReplaceOne { static constexpr auto name = "replaceOne"; };
|
||||||
struct NameReplaceAll { static const char * get() { return "replaceAll"; } };
|
struct NameReplaceAll { static constexpr auto name = "replaceAll"; };
|
||||||
struct NameReplaceRegexpOne { static const char * get() { return "replaceRegexpOne"; } };
|
struct NameReplaceRegexpOne { static constexpr auto name = "replaceRegexpOne"; };
|
||||||
struct NameReplaceRegexpAll { static const char * get() { return "replaceRegexpAll"; } };
|
struct NameReplaceRegexpAll { static constexpr auto name = "replaceRegexpAll"; };
|
||||||
|
|
||||||
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
|
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
|
||||||
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
|
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
|
||||||
|
@ -327,6 +327,11 @@ struct ExtractQueryStringAndFragment
|
|||||||
res_data = pos + (without_leading_char ? 1 : 0);
|
res_data = pos + (without_leading_char ? 1 : 0);
|
||||||
res_size = end - res_data;
|
res_size = end - res_data;
|
||||||
}
|
}
|
||||||
|
else if (nullptr != (pos = strchr(data, '#')))
|
||||||
|
{
|
||||||
|
res_data = pos;
|
||||||
|
res_size = end - res_data;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -503,7 +508,8 @@ private:
|
|||||||
bool first;
|
bool first;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static String getName() { return "extractURLParameters"; }
|
static constexpr auto name = "extractURLParameters";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
{
|
{
|
||||||
@ -571,7 +577,8 @@ private:
|
|||||||
bool first;
|
bool first;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static String getName() { return "extractURLParameterNames"; }
|
static constexpr auto name = "extractURLParameterNames";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
{
|
{
|
||||||
@ -639,7 +646,8 @@ private:
|
|||||||
Pos end;
|
Pos end;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static String getName() { return "URLHierarchy"; }
|
static constexpr auto name = "URLHierarchy";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
{
|
{
|
||||||
@ -737,7 +745,8 @@ private:
|
|||||||
Pos start;
|
Pos start;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static String getName() { return "URLPathHierarchy"; }
|
static constexpr auto name = "URLPathHierarchy";
|
||||||
|
static String getName() { return name; }
|
||||||
|
|
||||||
static void checkArguments(const DataTypes & arguments)
|
static void checkArguments(const DataTypes & arguments)
|
||||||
{
|
{
|
||||||
@ -924,25 +933,25 @@ struct CutSubstringImpl
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameProtocol { static const char * get() { return "protocol"; } };
|
struct NameProtocol { static constexpr auto name = "protocol"; };
|
||||||
struct NameDomain { static const char * get() { return "domain"; } };
|
struct NameDomain { static constexpr auto name = "domain"; };
|
||||||
struct NameDomainWithoutWWW { static const char * get() { return "domainWithoutWWW"; } };
|
struct NameDomainWithoutWWW { static constexpr auto name = "domainWithoutWWW"; };
|
||||||
struct NameFirstSignificantSubdomain { static const char * get() { return "firstSignificantSubdomain"; } };
|
struct NameFirstSignificantSubdomain { static constexpr auto name = "firstSignificantSubdomain"; };
|
||||||
struct NameTopLevelDomain { static const char * get() { return "topLevelDomain"; } };
|
struct NameTopLevelDomain { static constexpr auto name = "topLevelDomain"; };
|
||||||
struct NamePath { static const char * get() { return "path"; } };
|
struct NamePath { static constexpr auto name = "path"; };
|
||||||
struct NameQueryString { static const char * get() { return "queryString"; } };
|
struct NameQueryString { static constexpr auto name = "queryString"; };
|
||||||
struct NameFragment { static const char * get() { return "fragment"; } };
|
struct NameFragment { static constexpr auto name = "fragment"; };
|
||||||
struct NameQueryStringAndFragment { static const char * get() { return "queryStringAndFragment"; } };
|
struct NameQueryStringAndFragment { static constexpr auto name = "queryStringAndFragment"; };
|
||||||
|
|
||||||
struct NameCutToFirstSignificantSubdomain { static const char * get() { return "cutToFirstSignificantSubdomain"; } };
|
struct NameCutToFirstSignificantSubdomain { static constexpr auto name = "cutToFirstSignificantSubdomain"; };
|
||||||
|
|
||||||
struct NameCutWWW { static const char * get() { return "cutWWW"; } };
|
struct NameCutWWW { static constexpr auto name = "cutWWW"; };
|
||||||
struct NameCutQueryString { static const char * get() { return "cutQueryString"; } };
|
struct NameCutQueryString { static constexpr auto name = "cutQueryString"; };
|
||||||
struct NameCutFragment { static const char * get() { return "cutFragment"; } };
|
struct NameCutFragment { static constexpr auto name = "cutFragment"; };
|
||||||
struct NameCutQueryStringAndFragment { static const char * get() { return "cutQueryStringAndFragment"; } };
|
struct NameCutQueryStringAndFragment { static constexpr auto name = "cutQueryStringAndFragment"; };
|
||||||
|
|
||||||
struct NameExtractURLParameter { static const char * get() { return "extractURLParameter"; } };
|
struct NameExtractURLParameter { static constexpr auto name = "extractURLParameter"; };
|
||||||
struct NameCutURLParameter { static const char * get() { return "cutURLParameter"; } };
|
struct NameCutURLParameter { static constexpr auto name = "cutURLParameter"; };
|
||||||
|
|
||||||
typedef FunctionStringToString<ExtractSubstringImpl<ExtractProtocol>, NameProtocol> FunctionProtocol;
|
typedef FunctionStringToString<ExtractSubstringImpl<ExtractProtocol>, NameProtocol> FunctionProtocol;
|
||||||
typedef FunctionStringToString<ExtractSubstringImpl<ExtractDomain<false> >, NameDomain> FunctionDomain;
|
typedef FunctionStringToString<ExtractSubstringImpl<ExtractDomain<false> >, NameDomain> FunctionDomain;
|
||||||
|
@ -397,13 +397,13 @@ struct ExtractParamToStringImpl
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
struct NameVisitParamHas { static const char * get() { return "visitParamHas"; } };
|
struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; };
|
||||||
struct NameVisitParamExtractUInt { static const char * get() { return "visitParamExtractUInt"; } };
|
struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; };
|
||||||
struct NameVisitParamExtractInt { static const char * get() { return "visitParamExtractInt"; } };
|
struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; };
|
||||||
struct NameVisitParamExtractFloat { static const char * get() { return "visitParamExtractFloat"; } };
|
struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; };
|
||||||
struct NameVisitParamExtractBool { static const char * get() { return "visitParamExtractBool"; } };
|
struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; };
|
||||||
struct NameVisitParamExtractRaw { static const char * get() { return "visitParamExtractRaw"; } };
|
struct NameVisitParamExtractRaw { static constexpr auto name = "visitParamExtractRaw"; };
|
||||||
struct NameVisitParamExtractString { static const char * get() { return "visitParamExtractString"; } };
|
struct NameVisitParamExtractString { static constexpr auto name = "visitParamExtractString"; };
|
||||||
|
|
||||||
|
|
||||||
typedef FunctionsStringSearch<ExtractParamImpl<HasParam>, NameVisitParamHas> FunctionVisitParamHas;
|
typedef FunctionsStringSearch<ExtractParamImpl<HasParam>, NameVisitParamHas> FunctionVisitParamHas;
|
||||||
|
@ -210,9 +210,11 @@ private:
|
|||||||
*/
|
*/
|
||||||
void collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type);
|
void collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type);
|
||||||
|
|
||||||
|
void addStorageAliases();
|
||||||
|
|
||||||
/** Создать словарь алиасов.
|
/** Создать словарь алиасов.
|
||||||
*/
|
*/
|
||||||
void createAliasesDict(ASTPtr & ast, int ignore_levels = 0);
|
void addASTAliases(ASTPtr & ast, int ignore_levels = 0);
|
||||||
|
|
||||||
/** Для узлов-звёздочек - раскрыть их в список всех столбцов.
|
/** Для узлов-звёздочек - раскрыть их в список всех столбцов.
|
||||||
* Для узлов-литералов - подставить алиасы.
|
* Для узлов-литералов - подставить алиасы.
|
||||||
|
@ -22,7 +22,13 @@ public:
|
|||||||
|
|
||||||
/** Изменяет список столбцов в метаданных таблицы на диске. Нужно вызывать под TableStructureLock соответствующей таблицы.
|
/** Изменяет список столбцов в метаданных таблицы на диске. Нужно вызывать под TableStructureLock соответствующей таблицы.
|
||||||
*/
|
*/
|
||||||
static void updateMetadata(const String & database, const String & table, const NamesAndTypesList & columns, Context & context);
|
static void updateMetadata(const String & database,
|
||||||
|
const String & table,
|
||||||
|
const NamesAndTypesList & columns,
|
||||||
|
const NamesAndTypesList & materialized_columns,
|
||||||
|
const NamesAndTypesList & alias_columns,
|
||||||
|
const ColumnDefaults & column_defaults,
|
||||||
|
Context & context);
|
||||||
private:
|
private:
|
||||||
struct PartitionCommand
|
struct PartitionCommand
|
||||||
{
|
{
|
||||||
@ -74,4 +80,5 @@ private:
|
|||||||
static void parseAlter(const ASTAlterQuery::ParameterContainer & params, const DataTypeFactory & data_type_factory,
|
static void parseAlter(const ASTAlterQuery::ParameterContainer & params, const DataTypeFactory & data_type_factory,
|
||||||
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
|
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#include <DB/Storages/IStorage.h>
|
#include <DB/Storages/IStorage.h>
|
||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -22,12 +23,22 @@ public:
|
|||||||
*/
|
*/
|
||||||
StoragePtr execute(bool assume_metadata_exists = false);
|
StoragePtr execute(bool assume_metadata_exists = false);
|
||||||
|
|
||||||
/** AST в список столбцов с типами и обратно. Столбцы типа Nested развернуты в список настоящих столбцов.
|
/// Список столбцов с типами в AST.
|
||||||
*/
|
|
||||||
static NamesAndTypesList parseColumns(ASTPtr expression_list, const DataTypeFactory & data_type_factory);
|
|
||||||
static ASTPtr formatColumns(const NamesAndTypesList & columns);
|
static ASTPtr formatColumns(const NamesAndTypesList & columns);
|
||||||
|
static ASTPtr formatColumns(
|
||||||
|
NamesAndTypesList columns,
|
||||||
|
const NamesAndTypesList & materialized_columns,
|
||||||
|
const NamesAndTypesList & alias_columns,
|
||||||
|
const ColumnDefaults & column_defaults);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
/// AST в список столбцов с типами. Столбцы типа Nested развернуты в список настоящих столбцов.
|
||||||
|
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;
|
||||||
|
ColumnsAndDefaults parseColumns(ASTPtr expression_list);
|
||||||
|
|
||||||
|
/// removes columns from the columns list and return them in a separate list
|
||||||
|
static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_and_defaults, ColumnDefaultType type);
|
||||||
|
|
||||||
ASTPtr query_ptr;
|
ASTPtr query_ptr;
|
||||||
Context context;
|
Context context;
|
||||||
};
|
};
|
||||||
|
@ -10,6 +10,8 @@
|
|||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
|
#include <DB/Parsers/formatAST.h>
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -59,13 +61,18 @@ private:
|
|||||||
col.name = "name";
|
col.name = "name";
|
||||||
col.type = new DataTypeString;
|
col.type = new DataTypeString;
|
||||||
col.column = col.type->createColumn();
|
col.column = col.type->createColumn();
|
||||||
|
|
||||||
block.insert(col);
|
block.insert(col);
|
||||||
|
|
||||||
col.name = "type";
|
col.name = "type";
|
||||||
|
|
||||||
block.insert(col);
|
block.insert(col);
|
||||||
|
|
||||||
|
col.name = "default_type";
|
||||||
|
block.insert(col);
|
||||||
|
|
||||||
|
col.name = "default_expression";
|
||||||
|
block.insert(col);
|
||||||
|
|
||||||
|
|
||||||
return block;
|
return block;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -74,27 +81,42 @@ private:
|
|||||||
const ASTDescribeQuery & ast = typeid_cast<const ASTDescribeQuery &>(*query_ptr);
|
const ASTDescribeQuery & ast = typeid_cast<const ASTDescribeQuery &>(*query_ptr);
|
||||||
|
|
||||||
NamesAndTypesList columns;
|
NamesAndTypesList columns;
|
||||||
|
ColumnDefaults column_defaults;
|
||||||
|
|
||||||
{
|
{
|
||||||
StoragePtr table = context.getTable(ast.database, ast.table);
|
StoragePtr table = context.getTable(ast.database, ast.table);
|
||||||
auto table_lock = table->lockStructure(false);
|
auto table_lock = table->lockStructure(false);
|
||||||
columns = table->getColumnsList();
|
columns = table->getColumnsList();
|
||||||
|
columns.insert(std::end(columns), std::begin(table->alias_columns), std::end(table->alias_columns));
|
||||||
|
column_defaults = table->column_defaults;
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnString * name_column = new ColumnString;
|
ColumnWithNameAndType name_column{new ColumnString, new DataTypeString, "name"};
|
||||||
ColumnString * type_column = new ColumnString;
|
ColumnWithNameAndType type_column{new ColumnString, new DataTypeString, "type" };
|
||||||
|
ColumnWithNameAndType default_type_column{new ColumnString, new DataTypeString, "default_type" };
|
||||||
|
ColumnWithNameAndType default_expression_column{new ColumnString, new DataTypeString, "default_expression" };;
|
||||||
|
|
||||||
Block block;
|
for (const auto column : columns)
|
||||||
block.insert(ColumnWithNameAndType(name_column, new DataTypeString, "name"));
|
|
||||||
block.insert(ColumnWithNameAndType(type_column, new DataTypeString, "type"));
|
|
||||||
|
|
||||||
for (NamesAndTypesList::iterator it = columns.begin(); it != columns.end(); ++it)
|
|
||||||
{
|
{
|
||||||
name_column->insert(it->name);
|
name_column.column->insert(column.name);
|
||||||
type_column->insert(it->type->getName());
|
type_column.column->insert(column.type->getName());
|
||||||
|
|
||||||
|
const auto it = column_defaults.find(column.name);
|
||||||
|
if (it == std::end(column_defaults))
|
||||||
|
{
|
||||||
|
default_type_column.column->insertDefault();
|
||||||
|
default_expression_column.column->insertDefault();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
default_type_column.column->insert(toString(it->second.type));
|
||||||
|
default_expression_column.column->insert(queryToString(it->second.expression));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return new OneBlockInputStream(block);
|
return new OneBlockInputStream{
|
||||||
|
{name_column, type_column, default_type_column, default_expression_column}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
49
dbms/include/DB/Interpreters/evaluateMissingDefaults.h
Normal file
49
dbms/include/DB/Interpreters/evaluateMissingDefaults.h
Normal file
@ -0,0 +1,49 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Core/Block.h>
|
||||||
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
|
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||||
|
#include <utility>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
inline void evaluateMissingDefaults(Block & block,
|
||||||
|
const NamesAndTypesList & required_columns,
|
||||||
|
const ColumnDefaults & column_defaults,
|
||||||
|
const Context & context)
|
||||||
|
{
|
||||||
|
if (column_defaults.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
ASTPtr default_expr_list{stdext::make_unique<ASTExpressionList>().release()};
|
||||||
|
|
||||||
|
for (const auto & column : required_columns)
|
||||||
|
{
|
||||||
|
if (block.has(column.name))
|
||||||
|
continue;
|
||||||
|
|
||||||
|
const auto it = column_defaults.find(column.name);
|
||||||
|
|
||||||
|
/// expressions must be cloned to prevent modification by the ExpressionAnalyzer
|
||||||
|
if (it != column_defaults.end())
|
||||||
|
default_expr_list->children.emplace_back(
|
||||||
|
setAlias(it->second.expression->clone(), it->first));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// nothing to evaluate
|
||||||
|
if (default_expr_list->children.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
/** ExpressionAnalyzer eliminates "unused" columns, in order to ensure their safety
|
||||||
|
* we are going to operate on a copy instead of the original block */
|
||||||
|
Block copy_block{block};
|
||||||
|
/// evaluate default values for defaulted columns
|
||||||
|
ExpressionAnalyzer{default_expr_list, context, required_columns}.getActions(true)->execute(copy_block);
|
||||||
|
|
||||||
|
/// move evaluated columns to the original block
|
||||||
|
for (auto & column_name_type : copy_block.getColumns())
|
||||||
|
block.insert(std::move(column_name_type));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -38,7 +38,7 @@ public:
|
|||||||
* В запросе DROP это поле не используется
|
* В запросе DROP это поле не используется
|
||||||
* В запросе MODIFY здесь хранится имя столбца и новый тип
|
* В запросе MODIFY здесь хранится имя столбца и новый тип
|
||||||
*/
|
*/
|
||||||
ASTPtr name_type;
|
ASTPtr col_decl;
|
||||||
|
|
||||||
/** В запросе ADD COLUMN здесь опционально хранится имя столбца, следующее после AFTER
|
/** В запросе ADD COLUMN здесь опционально хранится имя столбца, следующее после AFTER
|
||||||
* В запросе DROP здесь хранится имя столбца для удаления
|
* В запросе DROP здесь хранится имя столбца для удаления
|
||||||
@ -61,7 +61,7 @@ public:
|
|||||||
void clone(Parameters & p) const
|
void clone(Parameters & p) const
|
||||||
{
|
{
|
||||||
p = *this;
|
p = *this;
|
||||||
p.name_type = name_type->clone();
|
p.col_decl = col_decl->clone();
|
||||||
p.column = column->clone();
|
p.column = column->clone();
|
||||||
p.partition = partition->clone();
|
p.partition = partition->clone();
|
||||||
}
|
}
|
||||||
@ -75,8 +75,8 @@ public:
|
|||||||
void addParameters(const Parameters & params)
|
void addParameters(const Parameters & params)
|
||||||
{
|
{
|
||||||
parameters.push_back(params);
|
parameters.push_back(params);
|
||||||
if (params.name_type)
|
if (params.col_decl)
|
||||||
children.push_back(params.name_type);
|
children.push_back(params.col_decl);
|
||||||
if (params.column)
|
if (params.column)
|
||||||
children.push_back(params.column);
|
children.push_back(params.column);
|
||||||
if (params.partition)
|
if (params.partition)
|
||||||
|
43
dbms/include/DB/Parsers/ASTColumnDeclaration.h
Normal file
43
dbms/include/DB/Parsers/ASTColumnDeclaration.h
Normal file
@ -0,0 +1,43 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Parsers/IAST.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/** Name, type, default-specifier, default-expression.
|
||||||
|
* The type is optional if default-expression is specified.
|
||||||
|
*/
|
||||||
|
class ASTColumnDeclaration : public IAST
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
String name;
|
||||||
|
ASTPtr type;
|
||||||
|
String default_specifier;
|
||||||
|
ASTPtr default_expression;
|
||||||
|
|
||||||
|
ASTColumnDeclaration() = default;
|
||||||
|
ASTColumnDeclaration(StringRange range) : IAST{range} {}
|
||||||
|
|
||||||
|
String getID() const { return "ColumnDeclaration_" + name; }
|
||||||
|
|
||||||
|
ASTPtr clone() const
|
||||||
|
{
|
||||||
|
const auto res = new ASTColumnDeclaration{*this};
|
||||||
|
res->children.clear();
|
||||||
|
|
||||||
|
if (type) {
|
||||||
|
res->type = type->clone();
|
||||||
|
res->children.push_back(res->type);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (default_expression) {
|
||||||
|
res->default_expression = default_expression->clone();
|
||||||
|
res->children.push_back(res->default_expression);
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Parsers/ASTWithAlias.h>
|
#include <DB/Parsers/ASTWithAlias.h>
|
||||||
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||||
#include <DB/IO/WriteBufferFromString.h>
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
@ -81,4 +82,37 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <typename... Args>
|
||||||
|
ASTPtr makeASTFunction(const String & name, Args &&... args)
|
||||||
|
{
|
||||||
|
const auto function = new ASTFunction{};
|
||||||
|
ASTPtr result{function};
|
||||||
|
|
||||||
|
function->name = name;
|
||||||
|
function->arguments = new ASTExpressionList{};
|
||||||
|
function->children.push_back(function->arguments);
|
||||||
|
|
||||||
|
function->arguments->children = { std::forward<Args>(args)... };
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename... Args>
|
||||||
|
ASTPtr makeASTFunction(const String & name, const StringRange & function_range,
|
||||||
|
const StringRange & arguments_range, Args &&... args)
|
||||||
|
{
|
||||||
|
const auto function = new ASTFunction{function_range};
|
||||||
|
ASTPtr result{function};
|
||||||
|
|
||||||
|
function->name = name;
|
||||||
|
function->arguments = new ASTExpressionList{arguments_range};
|
||||||
|
function->children.push_back(function->arguments);
|
||||||
|
|
||||||
|
function->arguments->children = { std::forward<Args>(args)... };
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -21,4 +21,12 @@ public:
|
|||||||
void setAlias(const String & to) override { alias = to; }
|
void setAlias(const String & to) override { alias = to; }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// helper for setting aliases and chaining result to other functions
|
||||||
|
inline ASTPtr setAlias(ASTPtr ast, const String & alias) {
|
||||||
|
dynamic_cast<ASTWithAlias &>(*ast).alias = alias;
|
||||||
|
|
||||||
|
return ast;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,9 +2,12 @@
|
|||||||
|
|
||||||
#include <DB/Parsers/IParserBase.h>
|
#include <DB/Parsers/IParserBase.h>
|
||||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||||
|
#include <DB/Parsers/ExpressionListParsers.h>
|
||||||
#include <DB/Parsers/ASTNameTypePair.h>
|
#include <DB/Parsers/ASTNameTypePair.h>
|
||||||
|
#include <DB/Parsers/ASTColumnDeclaration.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/CommonParsers.h>
|
#include <DB/Parsers/CommonParsers.h>
|
||||||
|
#include <Poco/String.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -93,6 +96,101 @@ protected:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <class NameParser>
|
||||||
|
class IParserColumnDeclaration : public IParserBase
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
const char * getName() const { return "column declaration"; }
|
||||||
|
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
|
||||||
|
};
|
||||||
|
|
||||||
|
typedef IParserColumnDeclaration<ParserIdentifier> ParserColumnDeclaration;
|
||||||
|
typedef IParserColumnDeclaration<ParserCompoundIdentifier> ParserCompoundColumnDeclaration;
|
||||||
|
|
||||||
|
template <class NameParser>
|
||||||
|
bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
NameParser name_parser;
|
||||||
|
ParserIdentifierWithOptionalParameters type_parser;
|
||||||
|
ParserWhiteSpaceOrComments ws;
|
||||||
|
ParserString s_default{"DEFAULT", true, true};
|
||||||
|
ParserString s_materialized{"MATERIALIZED", true, true};
|
||||||
|
ParserString s_alias{"ALIAS", true, true};
|
||||||
|
ParserTernaryOperatorExpression expr_parser;
|
||||||
|
|
||||||
|
const auto begin = pos;
|
||||||
|
const auto reset_pos_and_return = [&pos, begin] {
|
||||||
|
pos = begin;
|
||||||
|
return false;
|
||||||
|
};
|
||||||
|
|
||||||
|
/// mandatory column name
|
||||||
|
ASTPtr name;
|
||||||
|
if (!name_parser.parse(pos, end, name, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
ws.ignore(pos, end, expected);
|
||||||
|
|
||||||
|
/** column name should be followed by type name if it
|
||||||
|
* is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS} */
|
||||||
|
ASTPtr type;
|
||||||
|
const auto fallback_pos = pos;
|
||||||
|
if (!s_default.check(pos, end, expected) &&
|
||||||
|
!s_materialized.check(pos, end, expected) &&
|
||||||
|
!s_alias.check(pos, end, expected))
|
||||||
|
{
|
||||||
|
if (type_parser.parse(pos, end, type, expected))
|
||||||
|
ws.ignore(pos, end, expected);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
pos = fallback_pos;
|
||||||
|
|
||||||
|
/// parse {DEFAULT, MATERIALIZED, ALIAS}
|
||||||
|
String default_specifier;
|
||||||
|
ASTPtr default_expression;
|
||||||
|
const auto pos_before_specifier = pos;
|
||||||
|
if (s_default.ignore(pos, end, expected) ||
|
||||||
|
s_materialized.ignore(pos, end, expected) ||
|
||||||
|
s_alias.ignore(pos, end, expected))
|
||||||
|
{
|
||||||
|
default_specifier = Poco::toUpper(std::string{pos_before_specifier, pos});
|
||||||
|
|
||||||
|
/// should be followed by an expression
|
||||||
|
ws.ignore(pos, end, expected);
|
||||||
|
|
||||||
|
if (!expr_parser.parse(pos, end, default_expression, expected))
|
||||||
|
return reset_pos_and_return();
|
||||||
|
}
|
||||||
|
else if (!type)
|
||||||
|
return reset_pos_and_return(); /// reject sole column name without type
|
||||||
|
|
||||||
|
const auto column_declaration = new ASTColumnDeclaration{StringRange{begin, pos}};
|
||||||
|
node = column_declaration;
|
||||||
|
column_declaration->name = typeid_cast<ASTIdentifier &>(*name).name;
|
||||||
|
if (type)
|
||||||
|
{
|
||||||
|
column_declaration->type = type;
|
||||||
|
column_declaration->children.push_back(std::move(type));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (default_expression)
|
||||||
|
{
|
||||||
|
column_declaration->default_specifier = default_specifier;
|
||||||
|
column_declaration->default_expression = default_expression;
|
||||||
|
column_declaration->children.push_back(std::move(default_expression));
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
class ParserColumnDeclarationList : public IParserBase
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
const char * getName() const { return "column declaration list"; }
|
||||||
|
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
/** ENGINE = name. */
|
/** ENGINE = name. */
|
||||||
class ParserEngine : public IParserBase
|
class ParserEngine : public IParserBase
|
||||||
{
|
{
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ASTLiteral.h>
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
#include <DB/Parsers/ASTNameTypePair.h>
|
#include <DB/Parsers/ASTNameTypePair.h>
|
||||||
|
#include <DB/Parsers/ASTColumnDeclaration.h>
|
||||||
#include <DB/Parsers/ASTAsterisk.h>
|
#include <DB/Parsers/ASTAsterisk.h>
|
||||||
#include <DB/Parsers/ASTOrderByElement.h>
|
#include <DB/Parsers/ASTOrderByElement.h>
|
||||||
#include <DB/Parsers/ASTSubquery.h>
|
#include <DB/Parsers/ASTSubquery.h>
|
||||||
@ -54,6 +55,7 @@ void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, b
|
|||||||
void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
|
void formatAST(const ASTColumnDeclaration & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||||
|
8
dbms/include/DB/Parsers/queryToString.h
Normal file
8
dbms/include/DB/Parsers/queryToString.h
Normal file
@ -0,0 +1,8 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Parsers/IAST.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
String queryToString(const ASTPtr & query);
|
||||||
|
}
|
@ -1,7 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Core/NamesAndTypes.h>
|
#include <DB/Core/NamesAndTypes.h>
|
||||||
#include <DB/DataTypes/DataTypeNested.h>
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -23,6 +23,9 @@ struct AlterCommand
|
|||||||
/// Для ADD и MODIFY - новый тип столбца.
|
/// Для ADD и MODIFY - новый тип столбца.
|
||||||
DataTypePtr data_type;
|
DataTypePtr data_type;
|
||||||
|
|
||||||
|
ColumnDefaultType default_type{};
|
||||||
|
ASTPtr default_expression{};
|
||||||
|
|
||||||
/// Для ADD - после какого столбца добавить новый. Если пустая строка, добавить в конец. Добавить в начало сейчас нельзя.
|
/// Для ADD - после какого столбца добавить новый. Если пустая строка, добавить в конец. Добавить в начало сейчас нельзя.
|
||||||
String after_column;
|
String after_column;
|
||||||
|
|
||||||
@ -34,87 +37,32 @@ struct AlterCommand
|
|||||||
return (name_with_dot == name_type.name.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.name);
|
return (name_with_dot == name_type.name.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.name);
|
||||||
}
|
}
|
||||||
|
|
||||||
void apply(NamesAndTypesList & columns) const
|
void apply(NamesAndTypesList & columns,
|
||||||
{
|
NamesAndTypesList & materialized_columns,
|
||||||
if (type == ADD)
|
NamesAndTypesList & alias_columns,
|
||||||
{
|
ColumnDefaults & column_defaults) const;
|
||||||
if (std::count_if(columns.begin(), columns.end(), std::bind(namesEqual, column_name, std::placeholders::_1)))
|
|
||||||
throw Exception("Cannot add column " + column_name + ": column with this name already exisits.",
|
|
||||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
if (DataTypeNested::extractNestedTableName(column_name) != column_name &&
|
AlterCommand() = default;
|
||||||
!typeid_cast<const DataTypeArray *>(&*data_type))
|
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
|
||||||
throw Exception("Can't add nested column " + column_name + " of non-array type " + data_type->getName(),
|
const ColumnDefaultType default_type, const ASTPtr & default_expression,
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
const String & after_column = String{})
|
||||||
|
: type{type}, column_name{column_name}, data_type{data_type}, default_type{default_type},
|
||||||
NamesAndTypesList::iterator insert_it = columns.end();
|
default_expression{default_expression}, after_column{after_column}
|
||||||
if (!after_column.empty())
|
{}
|
||||||
{
|
|
||||||
/// Пытаемся найти первую с конца колонку с именем column_name или с именем, начинающимся с column_name и ".".
|
|
||||||
/// Например "fruits.bananas"
|
|
||||||
/// одинаковыми считаются имена, если они совпадают целиком или name_without_dot совпадает с частью имени до точки
|
|
||||||
NamesAndTypesList::reverse_iterator reverse_insert_it = std::find_if(columns.rbegin(), columns.rend(),
|
|
||||||
std::bind(namesEqual, after_column, std::placeholders::_1));
|
|
||||||
|
|
||||||
if (reverse_insert_it == columns.rend())
|
|
||||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to insert after",
|
|
||||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// base возвращает итератор, уже смещенный на один элемент вправо
|
|
||||||
insert_it = reverse_insert_it.base();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
columns.insert(insert_it, NameAndTypePair(column_name, data_type));
|
|
||||||
|
|
||||||
/// Медленно, так как каждый раз копируется список
|
|
||||||
columns = *DataTypeNested::expandNestedColumns(columns);
|
|
||||||
}
|
|
||||||
else if (type == DROP)
|
|
||||||
{
|
|
||||||
bool is_first = true;
|
|
||||||
NamesAndTypesList::iterator column_it;
|
|
||||||
do
|
|
||||||
{
|
|
||||||
column_it = std::find_if(columns.begin(), columns.end(), std::bind(namesEqual, column_name, std::placeholders::_1));
|
|
||||||
|
|
||||||
if (column_it == columns.end())
|
|
||||||
{
|
|
||||||
if (is_first)
|
|
||||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to drop",
|
|
||||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
columns.erase(column_it);
|
|
||||||
is_first = false;
|
|
||||||
}
|
|
||||||
while (column_it != columns.end());
|
|
||||||
}
|
|
||||||
else if (type == MODIFY)
|
|
||||||
{
|
|
||||||
NamesAndTypesList::iterator column_it = std::find_if(columns.begin(), columns.end(),
|
|
||||||
std::bind(namesEqual, column_name, std::placeholders::_1) );
|
|
||||||
if (column_it == columns.end())
|
|
||||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to modify.",
|
|
||||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
column_it->type = data_type;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class IStorage;
|
||||||
|
class Context;
|
||||||
|
|
||||||
class AlterCommands : public std::vector<AlterCommand>
|
class AlterCommands : public std::vector<AlterCommand>
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
void apply(NamesAndTypesList & columns) const
|
void apply(NamesAndTypesList & columns,
|
||||||
{
|
NamesAndTypesList & materialized_columns,
|
||||||
NamesAndTypesList new_columns = columns;
|
NamesAndTypesList & alias_columns,
|
||||||
for (const AlterCommand & command : *this)
|
ColumnDefaults & column_defaults) const;
|
||||||
command.apply(new_columns);
|
|
||||||
columns = new_columns;
|
void validate(IStorage * table, const Context & context);
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
178
dbms/include/DB/Storages/ColumnDefault.h
Normal file
178
dbms/include/DB/Storages/ColumnDefault.h
Normal file
@ -0,0 +1,178 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Parsers/IAST.h>
|
||||||
|
#include <DB/Parsers/queryToString.h>
|
||||||
|
#include <DB/Parsers/ExpressionListParsers.h>
|
||||||
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
enum struct ColumnDefaultType
|
||||||
|
{
|
||||||
|
Default,
|
||||||
|
Materialized,
|
||||||
|
Alias
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace std
|
||||||
|
{
|
||||||
|
template<> struct hash<DB::ColumnDefaultType>
|
||||||
|
{
|
||||||
|
size_t operator()(const DB::ColumnDefaultType type) const
|
||||||
|
{
|
||||||
|
return hash<int>{}(static_cast<int>(type));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
inline ColumnDefaultType columnDefaultTypeFromString(const String & str)
|
||||||
|
{
|
||||||
|
static const std::unordered_map<String, ColumnDefaultType> map{
|
||||||
|
{ "DEFAULT", ColumnDefaultType::Default },
|
||||||
|
{ "MATERIALIZED", ColumnDefaultType::Materialized },
|
||||||
|
{ "ALIAS", ColumnDefaultType::Alias }
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto it = map.find(str);
|
||||||
|
return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str};
|
||||||
|
}
|
||||||
|
|
||||||
|
inline String toString(const ColumnDefaultType type)
|
||||||
|
{
|
||||||
|
static const std::unordered_map<ColumnDefaultType, String> map{
|
||||||
|
{ ColumnDefaultType::Default, "DEFAULT" },
|
||||||
|
{ ColumnDefaultType::Materialized, "MATERIALIZED" },
|
||||||
|
{ ColumnDefaultType::Alias, "ALIAS" }
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto it = map.find(type);
|
||||||
|
return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultType"};
|
||||||
|
}
|
||||||
|
|
||||||
|
struct ColumnDefault
|
||||||
|
{
|
||||||
|
ColumnDefaultType type;
|
||||||
|
ASTPtr expression;
|
||||||
|
};
|
||||||
|
|
||||||
|
inline bool operator==(const ColumnDefault & lhs, const ColumnDefault & rhs)
|
||||||
|
{
|
||||||
|
return lhs.type == rhs.type && queryToString(lhs.expression) == queryToString(rhs.expression);
|
||||||
|
}
|
||||||
|
|
||||||
|
using ColumnDefaults = std::unordered_map<String, ColumnDefault>;
|
||||||
|
|
||||||
|
template <bool store>
|
||||||
|
struct ColumnsDescription
|
||||||
|
{
|
||||||
|
template <typename T> using by_value_or_cref = typename std::conditional<store, T, const T &>::type;
|
||||||
|
by_value_or_cref<NamesAndTypesList> columns;
|
||||||
|
by_value_or_cref<NamesAndTypesList> materialized;
|
||||||
|
by_value_or_cref<NamesAndTypesList> alias;
|
||||||
|
by_value_or_cref<ColumnDefaults> defaults;
|
||||||
|
|
||||||
|
String toString() const
|
||||||
|
{
|
||||||
|
String s;
|
||||||
|
WriteBufferFromString buf{s};
|
||||||
|
|
||||||
|
writeString("columns format version: 1\n", buf);
|
||||||
|
writeText(columns.size() + materialized.size() + alias.size(), buf);
|
||||||
|
writeString(" columns:\n", buf);
|
||||||
|
|
||||||
|
const auto write_columns = [this, &buf] (const NamesAndTypesList & columns) {
|
||||||
|
for (const auto & column : columns)
|
||||||
|
{
|
||||||
|
const auto it = defaults.find(column.name);
|
||||||
|
|
||||||
|
writeBackQuotedString(column.name, buf);
|
||||||
|
writeChar(' ', buf);
|
||||||
|
writeString(column.type->getName(), buf);
|
||||||
|
if (it == std::end(defaults))
|
||||||
|
{
|
||||||
|
writeChar('\n', buf);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
writeChar('\t', buf);
|
||||||
|
|
||||||
|
writeString(DB::toString(it->second.type), buf);
|
||||||
|
writeChar('\t', buf);
|
||||||
|
writeString(queryToString(it->second.expression), buf);
|
||||||
|
writeChar('\n', buf);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
write_columns(columns);
|
||||||
|
write_columns(materialized);
|
||||||
|
write_columns(alias);
|
||||||
|
|
||||||
|
return s;
|
||||||
|
}
|
||||||
|
|
||||||
|
static ColumnsDescription parse(const String & str, const DataTypeFactory & data_type_factory)
|
||||||
|
{
|
||||||
|
ReadBufferFromString buf{str};
|
||||||
|
|
||||||
|
assertString("columns format version: 1\n", buf);
|
||||||
|
size_t count{};
|
||||||
|
readText(count, buf);
|
||||||
|
assertString(" columns:\n", buf);
|
||||||
|
|
||||||
|
ParserTernaryOperatorExpression expr_parser;
|
||||||
|
|
||||||
|
ColumnsDescription result{};
|
||||||
|
for (size_t i = 0; i < count; ++i)
|
||||||
|
{
|
||||||
|
String column_name;
|
||||||
|
readBackQuotedString(column_name, buf);
|
||||||
|
assertString(" ", buf);
|
||||||
|
|
||||||
|
String type_name;
|
||||||
|
readString(type_name, buf);
|
||||||
|
auto type = data_type_factory.get(type_name);
|
||||||
|
if (*buf.position() == '\n')
|
||||||
|
{
|
||||||
|
assertString("\n", buf);
|
||||||
|
|
||||||
|
result.columns.emplace_back(column_name, std::move(type));
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
assertString("\t", buf);
|
||||||
|
|
||||||
|
String default_type_str;
|
||||||
|
readString(default_type_str, buf);
|
||||||
|
const auto default_type = columnDefaultTypeFromString(default_type_str);
|
||||||
|
assertString("\t", buf);
|
||||||
|
|
||||||
|
String default_expr_str;
|
||||||
|
readText(default_expr_str, buf);
|
||||||
|
assertString("\n", buf);
|
||||||
|
|
||||||
|
ASTPtr default_expr;
|
||||||
|
Expected expected{};
|
||||||
|
auto begin = default_expr_str.data();
|
||||||
|
const auto end = begin + default_expr_str.size();
|
||||||
|
if (!expr_parser.parse(begin, end, default_expr, expected))
|
||||||
|
throw Exception{"Could not parse default expression", DB::ErrorCodes::CANNOT_PARSE_TEXT};
|
||||||
|
|
||||||
|
if (ColumnDefaultType::Default == default_type)
|
||||||
|
result.columns.emplace_back(column_name, std::move(type));
|
||||||
|
else if (ColumnDefaultType::Materialized == default_type)
|
||||||
|
result.materialized.emplace_back(column_name, std::move(type));
|
||||||
|
else if (ColumnDefaultType::Alias == default_type)
|
||||||
|
result.alias.emplace_back(column_name, std::move(type));
|
||||||
|
|
||||||
|
result.defaults.emplace(column_name, ColumnDefault{default_type, default_expr});
|
||||||
|
}
|
||||||
|
|
||||||
|
assertEOF(buf);
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
@ -1,7 +1,8 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Storages/StorageDistributed.h>
|
#include <DB/Storages/StorageDistributed.h>
|
||||||
#include <DB/Storages/Distributed/queryToString.h>
|
|
||||||
|
#include <DB/Parsers/formatAST.h>
|
||||||
|
|
||||||
#include <DB/IO/WriteBufferFromFile.h>
|
#include <DB/IO/WriteBufferFromFile.h>
|
||||||
#include <DB/IO/CompressedWriteBuffer.h>
|
#include <DB/IO/CompressedWriteBuffer.h>
|
||||||
|
@ -1,14 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <DB/Parsers/formatAST.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
inline std::string queryToString(const ASTPtr & query)
|
|
||||||
{
|
|
||||||
std::ostringstream s;
|
|
||||||
formatAST(*query, s, 0, false, true);
|
|
||||||
|
|
||||||
return s.str();
|
|
||||||
}
|
|
||||||
}
|
|
@ -14,6 +14,7 @@
|
|||||||
#include <DB/Storages/AlterCommands.h>
|
#include <DB/Storages/AlterCommands.h>
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
#include <Poco/RWLock.h>
|
#include <Poco/RWLock.h>
|
||||||
|
#include <statdaemons/stdext.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -107,8 +108,8 @@ public:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
typedef Poco::SharedPtr<Poco::ScopedWriteRWLock> TableStructureWriteLockPtr;
|
typedef std::unique_ptr<Poco::ScopedWriteRWLock> TableStructureWriteLockPtr;
|
||||||
typedef Poco::SharedPtr<Poco::ScopedWriteRWLock> TableDataWriteLockPtr;
|
typedef std::unique_ptr<Poco::ScopedWriteRWLock> TableDataWriteLockPtr;
|
||||||
typedef std::pair<TableDataWriteLockPtr, TableStructureWriteLockPtr> TableFullWriteLockPtr;
|
typedef std::pair<TableDataWriteLockPtr, TableStructureWriteLockPtr> TableFullWriteLockPtr;
|
||||||
|
|
||||||
/** Не дает читать структуру таблицы. Берется для ALTER, RENAME и DROP.
|
/** Не дает читать структуру таблицы. Берется для ALTER, RENAME и DROP.
|
||||||
@ -124,7 +125,7 @@ public:
|
|||||||
*/
|
*/
|
||||||
TableDataWriteLockPtr lockDataForAlter()
|
TableDataWriteLockPtr lockDataForAlter()
|
||||||
{
|
{
|
||||||
TableDataWriteLockPtr res = new Poco::ScopedWriteRWLock(data_lock);
|
auto res = stdext::make_unique<Poco::ScopedWriteRWLock>(data_lock);
|
||||||
if (is_dropped)
|
if (is_dropped)
|
||||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||||
return res;
|
return res;
|
||||||
@ -132,7 +133,7 @@ public:
|
|||||||
|
|
||||||
TableStructureWriteLockPtr lockStructureForAlter()
|
TableStructureWriteLockPtr lockStructureForAlter()
|
||||||
{
|
{
|
||||||
TableStructureWriteLockPtr res = new Poco::ScopedWriteRWLock(structure_lock);
|
auto res = stdext::make_unique<Poco::ScopedWriteRWLock>(structure_lock);
|
||||||
if (is_dropped)
|
if (is_dropped)
|
||||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||||
return res;
|
return res;
|
||||||
@ -271,7 +272,7 @@ public:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool is_dropped;
|
bool is_dropped{false};
|
||||||
|
|
||||||
/// Поддерживается ли индекс в секции IN
|
/// Поддерживается ли индекс в секции IN
|
||||||
virtual bool supportsIndexForIn() const { return false; };
|
virtual bool supportsIndexForIn() const { return false; };
|
||||||
@ -280,7 +281,7 @@ public:
|
|||||||
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }
|
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
IStorage() : is_dropped(false) {}
|
using ITableDeclaration::ITableDeclaration;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
std::weak_ptr<IStorage> this_ptr;
|
std::weak_ptr<IStorage> this_ptr;
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DB/Core/NamesAndTypes.h>
|
#include <DB/Core/NamesAndTypes.h>
|
||||||
#include <DB/Core/Exception.h>
|
#include <DB/Core/Exception.h>
|
||||||
#include <DB/Core/Block.h>
|
#include <DB/Core/Block.h>
|
||||||
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -22,7 +23,8 @@ public:
|
|||||||
|
|
||||||
/** Получить список имён и типов столбцов таблицы, только невиртуальные.
|
/** Получить список имён и типов столбцов таблицы, только невиртуальные.
|
||||||
*/
|
*/
|
||||||
virtual const NamesAndTypesList & getColumnsList() const = 0;
|
NamesAndTypesList getColumnsList() const;
|
||||||
|
const NamesAndTypesList & getColumnsListNonMaterialized() const { return getColumnsListImpl(); }
|
||||||
|
|
||||||
/** Получить список имён столбцов таблицы, только невиртуальные.
|
/** Получить список имён столбцов таблицы, только невиртуальные.
|
||||||
*/
|
*/
|
||||||
@ -36,6 +38,9 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual bool hasRealColumn(const String & column_name) const;
|
virtual bool hasRealColumn(const String & column_name) const;
|
||||||
|
|
||||||
|
NameAndTypePair getMaterializedColumn(const String & column_name) const;
|
||||||
|
bool hasMaterializedColumn(const String & column_name) const;
|
||||||
|
|
||||||
/** Получить описание любого столбца по его имени.
|
/** Получить описание любого столбца по его имени.
|
||||||
*/
|
*/
|
||||||
virtual NameAndTypePair getColumn(const String & column_name) const;
|
virtual NameAndTypePair getColumn(const String & column_name) const;
|
||||||
@ -49,6 +54,7 @@ public:
|
|||||||
/** То же самое, но в виде блока-образца.
|
/** То же самое, но в виде блока-образца.
|
||||||
*/
|
*/
|
||||||
Block getSampleBlock() const;
|
Block getSampleBlock() const;
|
||||||
|
Block getSampleBlockNonMaterialized() const;
|
||||||
|
|
||||||
/** Проверить, что все запрошенные имена есть в таблице и заданы корректно.
|
/** Проверить, что все запрошенные имена есть в таблице и заданы корректно.
|
||||||
* (список имён не пустой и имена не повторяются)
|
* (список имён не пустой и имена не повторяются)
|
||||||
@ -69,7 +75,25 @@ public:
|
|||||||
*/
|
*/
|
||||||
void check(const Block & block, bool need_all = false) const;
|
void check(const Block & block, bool need_all = false) const;
|
||||||
|
|
||||||
virtual ~ITableDeclaration() {}
|
|
||||||
|
virtual ~ITableDeclaration() = default;
|
||||||
|
|
||||||
|
ITableDeclaration() = default;
|
||||||
|
ITableDeclaration(
|
||||||
|
const NamesAndTypesList & materialized_columns,
|
||||||
|
const NamesAndTypesList & alias_columns,
|
||||||
|
const ColumnDefaults & column_defaults)
|
||||||
|
: materialized_columns{materialized_columns},
|
||||||
|
alias_columns{alias_columns},
|
||||||
|
column_defaults{column_defaults}
|
||||||
|
{}
|
||||||
|
|
||||||
|
NamesAndTypesList materialized_columns{};
|
||||||
|
NamesAndTypesList alias_columns{};
|
||||||
|
ColumnDefaults column_defaults{};
|
||||||
|
|
||||||
|
private:
|
||||||
|
virtual const NamesAndTypesList & getColumnsListImpl() const = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -107,6 +107,47 @@ protected:
|
|||||||
/// Будем вызывать progressImpl самостоятельно.
|
/// Будем вызывать progressImpl самостоятельно.
|
||||||
void progress(const Progress & value) override {}
|
void progress(const Progress & value) override {}
|
||||||
|
|
||||||
|
void injectRequiredColumns(NamesAndTypesList & columns) const {
|
||||||
|
std::set<NameAndTypePair> required_columns;
|
||||||
|
auto modified = false;
|
||||||
|
|
||||||
|
for (auto it = std::begin(columns); it != std::end(columns);)
|
||||||
|
{
|
||||||
|
required_columns.emplace(*it);
|
||||||
|
|
||||||
|
if (!owned_data_part->hasColumnFiles(it->name))
|
||||||
|
{
|
||||||
|
const auto default_it = storage.column_defaults.find(it->name);
|
||||||
|
if (default_it != std::end(storage.column_defaults))
|
||||||
|
{
|
||||||
|
IdentifierNameSet identifiers;
|
||||||
|
default_it->second.expression->collectIdentifierNames(identifiers);
|
||||||
|
|
||||||
|
for (const auto & identifier : identifiers)
|
||||||
|
{
|
||||||
|
if (storage.hasColumn(identifier))
|
||||||
|
{
|
||||||
|
NameAndTypePair column{identifier, storage.getDataTypeByName(identifier)};
|
||||||
|
if (required_columns.count(column) == 0)
|
||||||
|
{
|
||||||
|
it = columns.emplace(++it, std::move(column));
|
||||||
|
modified = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (modified)
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
++it;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (modified)
|
||||||
|
columns = NamesAndTypesList{std::begin(required_columns), std::end(required_columns)};
|
||||||
|
}
|
||||||
|
|
||||||
Block readImpl()
|
Block readImpl()
|
||||||
{
|
{
|
||||||
Block res;
|
Block res;
|
||||||
@ -120,6 +161,9 @@ protected:
|
|||||||
/// NOTE В конструкторе это делать не получилось бы, потому что тогда ещё не установлен progress_callback.
|
/// NOTE В конструкторе это делать не получилось бы, потому что тогда ещё не установлен progress_callback.
|
||||||
progressImpl(Progress(0, 0, total_rows));
|
progressImpl(Progress(0, 0, total_rows));
|
||||||
|
|
||||||
|
injectRequiredColumns(columns);
|
||||||
|
injectRequiredColumns(pre_columns);
|
||||||
|
|
||||||
UncompressedCache * uncompressed_cache = use_uncompressed_cache ? storage.context.getUncompressedCache() : NULL;
|
UncompressedCache * uncompressed_cache = use_uncompressed_cache ? storage.context.getUncompressedCache() : NULL;
|
||||||
reader.reset(new MergeTreeReader(path, owned_data_part->name, columns, uncompressed_cache, storage, all_mark_ranges));
|
reader.reset(new MergeTreeReader(path, owned_data_part->name, columns, uncompressed_cache, storage, all_mark_ranges));
|
||||||
if (prewhere_actions)
|
if (prewhere_actions)
|
||||||
|
@ -585,6 +585,9 @@ public:
|
|||||||
* require_part_metadata - обязательно ли в директории с куском должны быть checksums.txt и columns.txt
|
* require_part_metadata - обязательно ли в директории с куском должны быть checksums.txt и columns.txt
|
||||||
*/
|
*/
|
||||||
MergeTreeData( const String & full_path_, NamesAndTypesListPtr columns_,
|
MergeTreeData( const String & full_path_, NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
ASTPtr & primary_expr_ast_,
|
ASTPtr & primary_expr_ast_,
|
||||||
const String & date_column_name_,
|
const String & date_column_name_,
|
||||||
@ -613,7 +616,7 @@ public:
|
|||||||
throw Exception("Logical error: calling method getTableName of not a table.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: calling method getTableName of not a table.", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
|
|
||||||
NameAndTypePair getColumn(const String & column_name) const
|
NameAndTypePair getColumn(const String & column_name) const
|
||||||
{
|
{
|
||||||
@ -621,14 +624,16 @@ public:
|
|||||||
return NameAndTypePair("_part", new DataTypeString);
|
return NameAndTypePair("_part", new DataTypeString);
|
||||||
if (column_name == "_part_index")
|
if (column_name == "_part_index")
|
||||||
return NameAndTypePair("_part_index", new DataTypeUInt64);
|
return NameAndTypePair("_part_index", new DataTypeUInt64);
|
||||||
return getRealColumn(column_name);
|
return ITableDeclaration::getColumn(column_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool hasColumn(const String & column_name) const
|
bool hasColumn(const String & column_name) const
|
||||||
{
|
{
|
||||||
if (column_name == "_part") return true;
|
if (column_name == "_part")
|
||||||
if (column_name == "_part_index") return true;
|
return true;
|
||||||
return hasRealColumn(column_name);
|
if (column_name == "_part_index")
|
||||||
|
return true;
|
||||||
|
return ITableDeclaration::hasColumn(column_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
String getFullPath() const { return full_path; }
|
String getFullPath() const { return full_path; }
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <DB/IO/CompressedReadBufferFromFile.h>
|
#include <DB/IO/CompressedReadBufferFromFile.h>
|
||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/Columns/ColumnNested.h>
|
#include <DB/Columns/ColumnNested.h>
|
||||||
|
#include <DB/Interpreters/evaluateMissingDefaults.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -124,7 +125,7 @@ public:
|
|||||||
storage.reportBrokenPart(part_name);
|
storage.reportBrokenPart(part_name);
|
||||||
|
|
||||||
/// Более хорошая диагностика.
|
/// Более хорошая диагностика.
|
||||||
throw Exception(e.message() + " (while reading from part " + path + " from mark " + toString(from_mark) + " to "
|
throw Exception(e.message() + "\n(while reading from part " + path + " from mark " + toString(from_mark) + " to "
|
||||||
+ toString(to_mark) + ")", e.code());
|
+ toString(to_mark) + ")", e.code());
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
@ -161,7 +162,8 @@ public:
|
|||||||
size_t pos = 0; /// Позиция, куда надо вставить недостающий столбец.
|
size_t pos = 0; /// Позиция, куда надо вставить недостающий столбец.
|
||||||
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it, ++pos)
|
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it, ++pos)
|
||||||
{
|
{
|
||||||
if (!res.has(it->name))
|
/// insert default values only for columns without default expressions
|
||||||
|
if (!res.has(it->name) && storage.column_defaults.count(it->name) == 0)
|
||||||
{
|
{
|
||||||
ColumnWithNameAndType column;
|
ColumnWithNameAndType column;
|
||||||
column.name = it->name;
|
column.name = it->name;
|
||||||
@ -192,11 +194,15 @@ public:
|
|||||||
res.insert(pos, column);
|
res.insert(pos, column);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// evaluate defaulted columns
|
||||||
|
evaluateMissingDefaults(res, columns, storage.column_defaults, storage.context);
|
||||||
}
|
}
|
||||||
catch (const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
/// Более хорошая диагностика.
|
/// Более хорошая диагностика.
|
||||||
throw Exception(e.message() + " (while reading from part " + path + ")", e.code());
|
throw Exception(e.message() + '\n' + e.getStackTrace().toString()
|
||||||
|
+ "\n(while reading from part " + path + ")", e.code());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -56,7 +56,7 @@ public:
|
|||||||
std::string getName() const override { return "Buffer"; }
|
std::string getName() const override { return "Buffer"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -21,6 +21,9 @@ public:
|
|||||||
const std::string & this_database_,/// Имя БД для этой таблицы.
|
const std::string & this_database_,/// Имя БД для этой таблицы.
|
||||||
const std::string & name_, /// Имя таблицы.
|
const std::string & name_, /// Имя таблицы.
|
||||||
NamesAndTypesListPtr columns_, /// Список столбцов.
|
NamesAndTypesListPtr columns_, /// Список столбцов.
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
const String & source_database_, /// В какой БД искать таблицы-источники.
|
const String & source_database_, /// В какой БД искать таблицы-источники.
|
||||||
const String & table_name_regexp_, /// Регексп имён таблиц-источников.
|
const String & table_name_regexp_, /// Регексп имён таблиц-источников.
|
||||||
const std::string & destination_name_prefix_, /// Префикс имен создаваемых таблиц типа Chunks.
|
const std::string & destination_name_prefix_, /// Префикс имен создаваемых таблиц типа Chunks.
|
||||||
@ -30,7 +33,7 @@ public:
|
|||||||
std::string getName() const override { return "ChunkMerger"; }
|
std::string getName() const override { return "ChunkMerger"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
NameAndTypePair getColumn(const String & column_name) const override;
|
NameAndTypePair getColumn(const String & column_name) const override;
|
||||||
bool hasColumn(const String & column_name) const override;
|
bool hasColumn(const String & column_name) const override;
|
||||||
|
|
||||||
@ -70,6 +73,9 @@ private:
|
|||||||
const std::string & this_database_,
|
const std::string & this_database_,
|
||||||
const std::string & name_,
|
const std::string & name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
const String & source_database_,
|
const String & source_database_,
|
||||||
const String & table_name_regexp_,
|
const String & table_name_regexp_,
|
||||||
const std::string & destination_name_prefix_,
|
const std::string & destination_name_prefix_,
|
||||||
|
@ -17,7 +17,7 @@ public:
|
|||||||
std::string getName() const override { return "ChunkRef"; }
|
std::string getName() const override { return "ChunkRef"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return getSource().getColumnsList(); }
|
const NamesAndTypesList & getColumnsListImpl() const override { return getSource().getColumnsListImpl(); }
|
||||||
/// В таблице, на которую мы ссылаемся, могут быть виртуальные столбцы.
|
/// В таблице, на которую мы ссылаемся, могут быть виртуальные столбцы.
|
||||||
NameAndTypePair getColumn(const String & column_name) const override { return getSource().getColumn(column_name); };
|
NameAndTypePair getColumn(const String & column_name) const override { return getSource().getColumn(column_name); };
|
||||||
bool hasColumn(const String & column_name) const override { return getSource().hasColumn(column_name); };
|
bool hasColumn(const String & column_name) const override { return getSource().hasColumn(column_name); };
|
||||||
|
@ -23,6 +23,9 @@ public:
|
|||||||
const std::string & name_,
|
const std::string & name_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
Context & context_,
|
Context & context_,
|
||||||
bool attach);
|
bool attach);
|
||||||
|
|
||||||
@ -93,6 +96,9 @@ private:
|
|||||||
const std::string & name_,
|
const std::string & name_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
Context & context_,
|
Context & context_,
|
||||||
bool attach);
|
bool attach);
|
||||||
|
|
||||||
|
@ -25,6 +25,9 @@ public:
|
|||||||
static StoragePtr create(
|
static StoragePtr create(
|
||||||
const std::string & name_, /// Имя таблицы.
|
const std::string & name_, /// Имя таблицы.
|
||||||
NamesAndTypesListPtr columns_, /// Список столбцов.
|
NamesAndTypesListPtr columns_, /// Список столбцов.
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
const String & remote_database_, /// БД на удалённых серверах.
|
const String & remote_database_, /// БД на удалённых серверах.
|
||||||
const String & remote_table_, /// Имя таблицы на удалённых серверах.
|
const String & remote_table_, /// Имя таблицы на удалённых серверах.
|
||||||
const String & cluster_name,
|
const String & cluster_name,
|
||||||
@ -46,7 +49,7 @@ public:
|
|||||||
bool supportsFinal() const override { return true; }
|
bool supportsFinal() const override { return true; }
|
||||||
bool supportsPrewhere() const override { return true; }
|
bool supportsPrewhere() const override { return true; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
NameAndTypePair getColumn(const String & column_name) const override;
|
NameAndTypePair getColumn(const String & column_name) const override;
|
||||||
bool hasColumn(const String & column_name) const override;
|
bool hasColumn(const String & column_name) const override;
|
||||||
|
|
||||||
@ -88,6 +91,19 @@ private:
|
|||||||
const ASTPtr & sharding_key_ = nullptr,
|
const ASTPtr & sharding_key_ = nullptr,
|
||||||
const String & data_path_ = String{});
|
const String & data_path_ = String{});
|
||||||
|
|
||||||
|
StorageDistributed(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
const String & remote_database_,
|
||||||
|
const String & remote_table_,
|
||||||
|
Cluster & cluster_,
|
||||||
|
Context & context_,
|
||||||
|
const ASTPtr & sharding_key_ = nullptr,
|
||||||
|
const String & data_path_ = String{});
|
||||||
|
|
||||||
|
|
||||||
/// create directory monitor thread by subdirectory name
|
/// create directory monitor thread by subdirectory name
|
||||||
void createDirectoryMonitor(const std::string & name);
|
void createDirectoryMonitor(const std::string & name);
|
||||||
|
@ -23,6 +23,9 @@ public:
|
|||||||
Context & context,
|
Context & context,
|
||||||
ASTPtr & query,
|
ASTPtr & query,
|
||||||
NamesAndTypesListPtr columns,
|
NamesAndTypesListPtr columns,
|
||||||
|
const NamesAndTypesList & materialized_columns,
|
||||||
|
const NamesAndTypesList & alias_columns,
|
||||||
|
const ColumnDefaults & column_defaults,
|
||||||
bool attach) const;
|
bool attach) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -139,6 +139,15 @@ public:
|
|||||||
* (корректность имён и путей не проверяется)
|
* (корректность имён и путей не проверяется)
|
||||||
* состоящую из указанных столбцов; создать файлы, если их нет.
|
* состоящую из указанных столбцов; создать файлы, если их нет.
|
||||||
*/
|
*/
|
||||||
|
static StoragePtr create(
|
||||||
|
const std::string & path_,
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
|
||||||
|
|
||||||
static StoragePtr create(
|
static StoragePtr create(
|
||||||
const std::string & path_,
|
const std::string & path_,
|
||||||
const std::string & name_,
|
const std::string & name_,
|
||||||
@ -148,7 +157,7 @@ public:
|
|||||||
std::string getName() const override { return "Log"; }
|
std::string getName() const override { return "Log"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
|
|
||||||
virtual BlockInputStreams read(
|
virtual BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
@ -196,7 +205,14 @@ protected:
|
|||||||
throw Exception("There is no column " + _table_column_name + " in table " + getTableName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
throw Exception("There is no column " + _table_column_name + " in table " + getTableName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||||
}
|
}
|
||||||
|
|
||||||
StorageLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, size_t max_compress_block_size_);
|
StorageLog(
|
||||||
|
const std::string & path_,
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
size_t max_compress_block_size_);
|
||||||
|
|
||||||
/// Прочитать файлы с засечками, если они ещё не прочитаны.
|
/// Прочитать файлы с засечками, если они ещё не прочитаны.
|
||||||
/// Делается лениво, чтобы при большом количестве таблиц, сервер быстро стартовал.
|
/// Делается лениво, чтобы при большом количестве таблиц, сервер быстро стартовал.
|
||||||
|
@ -9,8 +9,16 @@ namespace DB
|
|||||||
class StorageMaterializedView : public StorageView {
|
class StorageMaterializedView : public StorageView {
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static StoragePtr create(const String & table_name_, const String & database_name_,
|
static StoragePtr create(
|
||||||
Context & context_, ASTPtr & query_, NamesAndTypesListPtr columns_, bool attach_);
|
const String & table_name_,
|
||||||
|
const String & database_name_,
|
||||||
|
Context & context_,
|
||||||
|
ASTPtr & query_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
bool attach_);
|
||||||
|
|
||||||
std::string getName() const override { return "MaterializedView"; }
|
std::string getName() const override { return "MaterializedView"; }
|
||||||
std::string getInnerTableName() const { return ".inner." + table_name; }
|
std::string getInnerTableName() const { return ".inner." + table_name; }
|
||||||
@ -37,8 +45,16 @@ public:
|
|||||||
private:
|
private:
|
||||||
StoragePtr data;
|
StoragePtr data;
|
||||||
|
|
||||||
StorageMaterializedView(const String & table_name_, const String & database_name_,
|
StorageMaterializedView(
|
||||||
Context & context_, ASTPtr & query_, NamesAndTypesListPtr columns_, bool attach_);
|
const String & table_name_,
|
||||||
|
const String & database_name_,
|
||||||
|
Context & context_,
|
||||||
|
ASTPtr & query_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
bool attach_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -62,12 +62,21 @@ friend class MemoryBlockInputStream;
|
|||||||
friend class MemoryBlockOutputStream;
|
friend class MemoryBlockOutputStream;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_);
|
static StoragePtr create(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_);
|
||||||
|
|
||||||
|
static StoragePtr create(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_);
|
||||||
|
|
||||||
std::string getName() const override { return "Memory"; }
|
std::string getName() const override { return "Memory"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
|
|
||||||
size_t getSize() const { return data.size(); }
|
size_t getSize() const { return data.size(); }
|
||||||
|
|
||||||
@ -93,7 +102,16 @@ private:
|
|||||||
|
|
||||||
Poco::FastMutex mutex;
|
Poco::FastMutex mutex;
|
||||||
|
|
||||||
StorageMemory(const std::string & name_, NamesAndTypesListPtr columns_);
|
StorageMemory(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_);
|
||||||
|
|
||||||
|
StorageMemory(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -25,6 +25,16 @@ public:
|
|||||||
const String & table_name_regexp_, /// Регексп имён таблиц-источников.
|
const String & table_name_regexp_, /// Регексп имён таблиц-источников.
|
||||||
const Context & context_); /// Известные таблицы.
|
const Context & context_); /// Известные таблицы.
|
||||||
|
|
||||||
|
static StoragePtr create(
|
||||||
|
const std::string & name_, /// Имя таблицы.
|
||||||
|
NamesAndTypesListPtr columns_, /// Список столбцов.
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
const String & source_database_, /// В какой БД искать таблицы-источники.
|
||||||
|
const String & table_name_regexp_, /// Регексп имён таблиц-источников.
|
||||||
|
const Context & context_); /// Известные таблицы.
|
||||||
|
|
||||||
std::string getName() const override { return "Merge"; }
|
std::string getName() const override { return "Merge"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
bool supportsSampling() const override { return true; }
|
bool supportsSampling() const override { return true; }
|
||||||
@ -32,7 +42,7 @@ public:
|
|||||||
/// Проверка откладывается до метода read. Там проверяется поддержка PREWHERE у использующихся таблиц.
|
/// Проверка откладывается до метода read. Там проверяется поддержка PREWHERE у использующихся таблиц.
|
||||||
bool supportsPrewhere() const override { return true; }
|
bool supportsPrewhere() const override { return true; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
NameAndTypePair getColumn(const String &column_name) const override;
|
NameAndTypePair getColumn(const String &column_name) const override;
|
||||||
bool hasColumn(const String &column_name) const override;
|
bool hasColumn(const String &column_name) const override;
|
||||||
|
|
||||||
@ -65,6 +75,16 @@ private:
|
|||||||
const String & table_name_regexp_,
|
const String & table_name_regexp_,
|
||||||
const Context & context_);
|
const Context & context_);
|
||||||
|
|
||||||
|
StorageMerge(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
const String & source_database_,
|
||||||
|
const String & table_name_regexp_,
|
||||||
|
const Context & context_);
|
||||||
|
|
||||||
void getSelectedTables(StorageVector & selected_tables) const;
|
void getSelectedTables(StorageVector & selected_tables) const;
|
||||||
|
|
||||||
Block getBlockWithVirtualColumns(const StorageVector & selected_tables) const;
|
Block getBlockWithVirtualColumns(const StorageVector & selected_tables) const;
|
||||||
|
@ -24,7 +24,27 @@ public:
|
|||||||
* date_column_name - имя столбца с датой;
|
* date_column_name - имя столбца с датой;
|
||||||
* index_granularity - на сколько строчек пишется одно значение индекса.
|
* index_granularity - на сколько строчек пишется одно значение индекса.
|
||||||
*/
|
*/
|
||||||
static StoragePtr create(const String & path_, const String & database_name_, const String & table_name_,
|
static StoragePtr create(
|
||||||
|
const String & path_,
|
||||||
|
const String & database_name_,
|
||||||
|
const String & table_name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
Context & context_,
|
||||||
|
ASTPtr & primary_expr_ast_,
|
||||||
|
const String & date_column_name_,
|
||||||
|
const ASTPtr & sampling_expression_, /// nullptr, если семплирование не поддерживается.
|
||||||
|
size_t index_granularity_,
|
||||||
|
MergeTreeData::Mode mode_ = MergeTreeData::Ordinary,
|
||||||
|
const String & sign_column_ = "",
|
||||||
|
const MergeTreeSettings & settings_ = MergeTreeSettings());
|
||||||
|
|
||||||
|
static StoragePtr create(
|
||||||
|
const String & path_,
|
||||||
|
const String & database_name_,
|
||||||
|
const String & table_name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
Context & context_,
|
Context & context_,
|
||||||
ASTPtr & primary_expr_ast_,
|
ASTPtr & primary_expr_ast_,
|
||||||
@ -35,6 +55,7 @@ public:
|
|||||||
const String & sign_column_ = "",
|
const String & sign_column_ = "",
|
||||||
const MergeTreeSettings & settings_ = MergeTreeSettings());
|
const MergeTreeSettings & settings_ = MergeTreeSettings());
|
||||||
|
|
||||||
|
|
||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
~StorageMergeTree() override;
|
~StorageMergeTree() override;
|
||||||
|
|
||||||
@ -48,7 +69,7 @@ public:
|
|||||||
bool supportsFinal() const override { return data.supportsFinal(); }
|
bool supportsFinal() const override { return data.supportsFinal(); }
|
||||||
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
|
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return data.getColumnsList(); }
|
const NamesAndTypesList & getColumnsListImpl() const override { return data.getColumnsListNonMaterialized(); }
|
||||||
|
|
||||||
NameAndTypePair getColumn(const String & column_name) const override
|
NameAndTypePair getColumn(const String & column_name) const override
|
||||||
{
|
{
|
||||||
@ -157,8 +178,14 @@ private:
|
|||||||
|
|
||||||
typedef Poco::SharedPtr<CurrentlyMergingPartsTagger> CurrentlyMergingPartsTaggerPtr;
|
typedef Poco::SharedPtr<CurrentlyMergingPartsTagger> CurrentlyMergingPartsTaggerPtr;
|
||||||
|
|
||||||
StorageMergeTree(const String & path_, const String & database_name_, const String & table_name_,
|
StorageMergeTree(
|
||||||
|
const String & path_,
|
||||||
|
const String & database_name_,
|
||||||
|
const String & table_name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
Context & context_,
|
Context & context_,
|
||||||
ASTPtr & primary_expr_ast_,
|
ASTPtr & primary_expr_ast_,
|
||||||
const String & date_column_name_,
|
const String & date_column_name_,
|
||||||
|
@ -15,15 +15,20 @@ namespace DB
|
|||||||
class StorageNull : public IStorage
|
class StorageNull : public IStorage
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_)
|
static StoragePtr create(
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_)
|
||||||
{
|
{
|
||||||
return (new StorageNull(name_, columns_))->thisPtr();
|
return (new StorageNull{name_, columns_, materialized_columns_, alias_columns_, column_defaults_})->thisPtr();
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string getName() const override { return "Null"; }
|
std::string getName() const override { return "Null"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
@ -47,8 +52,13 @@ private:
|
|||||||
String name;
|
String name;
|
||||||
NamesAndTypesListPtr columns;
|
NamesAndTypesListPtr columns;
|
||||||
|
|
||||||
StorageNull(const std::string & name_, NamesAndTypesListPtr columns_)
|
StorageNull(
|
||||||
: name(name_), columns(columns_) {}
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_)
|
||||||
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_}, name(name_), columns(columns_) {}
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -31,6 +31,9 @@ public:
|
|||||||
bool attach,
|
bool attach,
|
||||||
const String & path_, const String & database_name_, const String & name_,
|
const String & path_, const String & database_name_, const String & name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
Context & context_,
|
Context & context_,
|
||||||
ASTPtr & primary_expr_ast_,
|
ASTPtr & primary_expr_ast_,
|
||||||
const String & date_column_name_,
|
const String & date_column_name_,
|
||||||
@ -53,7 +56,7 @@ public:
|
|||||||
bool supportsFinal() const override { return data.supportsFinal(); }
|
bool supportsFinal() const override { return data.supportsFinal(); }
|
||||||
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
|
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return data.getColumnsList(); }
|
const NamesAndTypesList & getColumnsListImpl() const override { return data.getColumnsListNonMaterialized(); }
|
||||||
|
|
||||||
NameAndTypePair getColumn(const String & column_name) const override
|
NameAndTypePair getColumn(const String & column_name) const override
|
||||||
{
|
{
|
||||||
@ -248,6 +251,9 @@ private:
|
|||||||
bool attach,
|
bool attach,
|
||||||
const String & path_, const String & database_name_, const String & name_,
|
const String & path_, const String & database_name_, const String & name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
Context & context_,
|
Context & context_,
|
||||||
ASTPtr & primary_expr_ast_,
|
ASTPtr & primary_expr_ast_,
|
||||||
const String & date_column_name_,
|
const String & date_column_name_,
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemDatabases"; }
|
std::string getName() const override { return "SystemDatabases"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -21,7 +21,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemEvents"; }
|
std::string getName() const override { return "SystemEvents"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -13,7 +13,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemMerges"; }
|
std::string getName() const override { return "SystemMerges"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
ASTPtr query,
|
ASTPtr query,
|
||||||
|
@ -23,7 +23,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemNumbers"; }
|
std::string getName() const override { return "SystemNumbers"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -20,7 +20,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemOne"; }
|
std::string getName() const override { return "SystemOne"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override{ return "SystemParts"; }
|
std::string getName() const override{ return "SystemParts"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemProcesses"; }
|
std::string getName() const override { return "SystemProcesses"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemReplicas"; }
|
std::string getName() const override { return "SystemReplicas"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemSettings"; }
|
std::string getName() const override { return "SystemSettings"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemTables"; }
|
std::string getName() const override { return "SystemTables"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
@ -38,6 +38,8 @@ private:
|
|||||||
NamesAndTypesList columns;
|
NamesAndTypesList columns;
|
||||||
|
|
||||||
StorageSystemTables(const std::string & name_, const Context & context_);
|
StorageSystemTables(const std::string & name_, const Context & context_);
|
||||||
|
|
||||||
|
ColumnWithNameAndType getFilteredDatabases(ASTPtr query);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -22,7 +22,7 @@ public:
|
|||||||
std::string getName() const override { return "SystemZooKeeper"; }
|
std::string getName() const override { return "SystemZooKeeper"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -115,13 +115,16 @@ public:
|
|||||||
const std::string & path_,
|
const std::string & path_,
|
||||||
const std::string & name_,
|
const std::string & name_,
|
||||||
NamesAndTypesListPtr columns_,
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
bool attach,
|
bool attach,
|
||||||
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
|
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
|
||||||
|
|
||||||
std::string getName() const override { return "TinyLog"; }
|
std::string getName() const override { return "TinyLog"; }
|
||||||
std::string getTableName() const override { return name; }
|
std::string getTableName() const override { return name; }
|
||||||
|
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
@ -163,7 +166,15 @@ private:
|
|||||||
|
|
||||||
Logger * log;
|
Logger * log;
|
||||||
|
|
||||||
StorageTinyLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, bool attach, size_t max_compress_block_size_);
|
StorageTinyLog(
|
||||||
|
const std::string & path_,
|
||||||
|
const std::string & name_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_,
|
||||||
|
bool attach,
|
||||||
|
size_t max_compress_block_size_);
|
||||||
|
|
||||||
void addFile(const String & column_name, const IDataType & type, size_t level = 0);
|
void addFile(const String & column_name, const IDataType & type, size_t level = 0);
|
||||||
};
|
};
|
||||||
|
@ -11,12 +11,19 @@ class StorageView : public IStorage
|
|||||||
{
|
{
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static StoragePtr create(const String & table_name_, const String & database_name_,
|
static StoragePtr create(
|
||||||
Context & context_, ASTPtr & query_, NamesAndTypesListPtr columns_);
|
const String & table_name_,
|
||||||
|
const String & database_name_,
|
||||||
|
Context & context_,
|
||||||
|
ASTPtr & query_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_);
|
||||||
|
|
||||||
std::string getName() const override { return "View"; }
|
std::string getName() const override { return "View"; }
|
||||||
std::string getTableName() const override { return table_name; }
|
std::string getTableName() const override { return table_name; }
|
||||||
const NamesAndTypesList & getColumnsList() const override { return *columns; }
|
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||||
ASTPtr getInnerQuery() const { return inner_query.clone(); };
|
ASTPtr getInnerQuery() const { return inner_query.clone(); };
|
||||||
|
|
||||||
/// Пробрасывается внутрь запроса и решается на его уровне.
|
/// Пробрасывается внутрь запроса и решается на его уровне.
|
||||||
@ -42,8 +49,15 @@ protected:
|
|||||||
Context & context;
|
Context & context;
|
||||||
NamesAndTypesListPtr columns;
|
NamesAndTypesListPtr columns;
|
||||||
|
|
||||||
StorageView(const String & table_name_, const String & database_name_,
|
StorageView(
|
||||||
Context & context_, ASTPtr & query_, NamesAndTypesListPtr columns_);
|
const String & table_name_,
|
||||||
|
const String & database_name_,
|
||||||
|
Context & context_,
|
||||||
|
ASTPtr & query_,
|
||||||
|
NamesAndTypesListPtr columns_,
|
||||||
|
const NamesAndTypesList & materialized_columns_,
|
||||||
|
const NamesAndTypesList & alias_columns_,
|
||||||
|
const ColumnDefaults & column_defaults_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -48,7 +48,8 @@ public:
|
|||||||
/// Нам необходимо его пометить как имя базы данных, поскольку по умолчанию стоит значение column
|
/// Нам необходимо его пометить как имя базы данных, поскольку по умолчанию стоит значение column
|
||||||
typeid_cast<ASTIdentifier &>(*args[0]).kind = ASTIdentifier::Database;
|
typeid_cast<ASTIdentifier &>(*args[0]).kind = ASTIdentifier::Database;
|
||||||
|
|
||||||
return StorageMerge::create(getName(), chooseColumns(source_database, table_name_regexp, context), source_database, table_name_regexp, context);
|
return StorageMerge::create(getName(), chooseColumns(source_database, table_name_regexp, context),
|
||||||
|
source_database, table_name_regexp, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -5,9 +5,17 @@
|
|||||||
|
|
||||||
#include <DB/Core/Block.h>
|
#include <DB/Core/Block.h>
|
||||||
|
|
||||||
|
#include <DB/Storages/ColumnDefault.h>
|
||||||
|
|
||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/DataTypes/DataTypeNested.h>
|
#include <DB/DataTypes/DataTypeNested.h>
|
||||||
|
|
||||||
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
|
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||||
|
#include <statdaemons/stdext.h>
|
||||||
|
|
||||||
|
#include <DB/Parsers/formatAST.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -19,20 +27,11 @@ Block::Block(const Block & other)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void Block::addDefaults(NamesAndTypesListPtr required_columns)
|
void Block::addDefaults(const NamesAndTypesList & required_columns)
|
||||||
{
|
{
|
||||||
for (NamesAndTypesList::const_iterator it = required_columns->begin(); it != required_columns->end(); ++it)
|
for (const auto & column : required_columns)
|
||||||
{
|
if (!has(column.name))
|
||||||
if (!has(it->name))
|
insertDefault(column.name, column.type);
|
||||||
{
|
|
||||||
ColumnWithNameAndType col;
|
|
||||||
col.name = it->name;
|
|
||||||
col.type = it->type;
|
|
||||||
col.column = dynamic_cast<IColumnConst &>(*it->type->createConstColumn(
|
|
||||||
rows(), it->type->getDefault())).convertToFullColumn();
|
|
||||||
insert(col);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Block & Block::operator= (const Block & other)
|
Block & Block::operator= (const Block & other)
|
||||||
@ -82,6 +81,16 @@ void Block::insert(const ColumnWithNameAndType & elem)
|
|||||||
index_by_position.push_back(it);
|
index_by_position.push_back(it);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Block::insertDefault(const String & name, const DataTypePtr & type)
|
||||||
|
{
|
||||||
|
insert({
|
||||||
|
dynamic_cast<IColumnConst &>(*type->createConstColumn(rows(),
|
||||||
|
type->getDefault())).convertToFullColumn(),
|
||||||
|
type, name
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
void Block::insertUnique(const ColumnWithNameAndType & elem)
|
void Block::insertUnique(const ColumnWithNameAndType & elem)
|
||||||
{
|
{
|
||||||
|
@ -6,23 +6,19 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsArithmetic(FunctionFactory & factory)
|
void registerFunctionsArithmetic(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionPlus>();
|
||||||
|
factory.registerFunction<FunctionMinus>();
|
||||||
factory.registerFunction("plus", F { return new FunctionPlus; });
|
factory.registerFunction<FunctionMultiply>();
|
||||||
factory.registerFunction("minus", F { return new FunctionMinus; });
|
factory.registerFunction<FunctionDivideFloating>();
|
||||||
factory.registerFunction("multiply", F { return new FunctionMultiply; });
|
factory.registerFunction<FunctionDivideIntegral>();
|
||||||
factory.registerFunction("divide", F { return new FunctionDivideFloating; });
|
factory.registerFunction<FunctionModulo>();
|
||||||
factory.registerFunction("intDiv", F { return new FunctionDivideIntegral; });
|
factory.registerFunction<FunctionNegate>();
|
||||||
factory.registerFunction("modulo", F { return new FunctionModulo; });
|
factory.registerFunction<FunctionBitAnd>();
|
||||||
factory.registerFunction("negate", F { return new FunctionNegate; });
|
factory.registerFunction<FunctionBitOr>();
|
||||||
factory.registerFunction("bitAnd", F { return new FunctionBitAnd; });
|
factory.registerFunction<FunctionBitXor>();
|
||||||
factory.registerFunction("bitOr", F { return new FunctionBitOr; });
|
factory.registerFunction<FunctionBitNot>();
|
||||||
factory.registerFunction("bitXor", F { return new FunctionBitXor; });
|
factory.registerFunction<FunctionBitShiftLeft>();
|
||||||
factory.registerFunction("bitNot", F { return new FunctionBitNot; });
|
factory.registerFunction<FunctionBitShiftRight>();
|
||||||
factory.registerFunction("bitShiftLeft", F { return new FunctionBitShiftLeft; });
|
|
||||||
factory.registerFunction("bitShiftRight", F { return new FunctionBitShiftRight; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,31 +6,26 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsArray(FunctionFactory & factory)
|
void registerFunctionsArray(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction *
|
factory.registerFunction<FunctionArray>();
|
||||||
|
factory.registerFunction<FunctionArrayElement>();
|
||||||
factory.registerFunction("array", F { return new FunctionArray; });
|
factory.registerFunction<FunctionHas>();
|
||||||
factory.registerFunction("arrayElement", F { return new FunctionArrayElement; });
|
factory.registerFunction<FunctionIndexOf>();
|
||||||
factory.registerFunction("has", F { return new FunctionHas; });
|
factory.registerFunction<FunctionCountEqual>();
|
||||||
factory.registerFunction("indexOf", F { return new FunctionIndexOf; });
|
factory.registerFunction<FunctionArrayEnumerate>();
|
||||||
factory.registerFunction("countEqual", F { return new FunctionCountEqual; });
|
factory.registerFunction<FunctionArrayEnumerateUniq>();
|
||||||
factory.registerFunction("arrayEnumerate", F { return new FunctionArrayEnumerate; });
|
factory.registerFunction<FunctionEmptyArrayUInt8>();
|
||||||
factory.registerFunction("arrayEnumerateUniq", F { return new FunctionArrayEnumerateUniq; });
|
factory.registerFunction<FunctionEmptyArrayUInt16>();
|
||||||
factory.registerFunction("emptyArrayUInt8", F { return new FunctionEmptyArrayUInt8; });
|
factory.registerFunction<FunctionEmptyArrayUInt32>();
|
||||||
factory.registerFunction("emptyArrayUInt16", F { return new FunctionEmptyArrayUInt16; });
|
factory.registerFunction<FunctionEmptyArrayUInt64>();
|
||||||
factory.registerFunction("emptyArrayUInt32", F { return new FunctionEmptyArrayUInt32; });
|
factory.registerFunction<FunctionEmptyArrayInt8>();
|
||||||
factory.registerFunction("emptyArrayUInt64", F { return new FunctionEmptyArrayUInt64; });
|
factory.registerFunction<FunctionEmptyArrayInt16>();
|
||||||
factory.registerFunction("emptyArrayInt8", F { return new FunctionEmptyArrayInt8; });
|
factory.registerFunction<FunctionEmptyArrayInt32>();
|
||||||
factory.registerFunction("emptyArrayInt16", F { return new FunctionEmptyArrayInt16; });
|
factory.registerFunction<FunctionEmptyArrayInt64>();
|
||||||
factory.registerFunction("emptyArrayInt32", F { return new FunctionEmptyArrayInt32; });
|
factory.registerFunction<FunctionEmptyArrayFloat32>();
|
||||||
factory.registerFunction("emptyArrayInt64", F { return new FunctionEmptyArrayInt64; });
|
factory.registerFunction<FunctionEmptyArrayFloat64>();
|
||||||
factory.registerFunction("emptyArrayFloat32", F { return new FunctionEmptyArrayFloat32; });
|
factory.registerFunction<FunctionEmptyArrayDate>();
|
||||||
factory.registerFunction("emptyArrayFloat64", F { return new FunctionEmptyArrayFloat64; });
|
factory.registerFunction<FunctionEmptyArrayDateTime>();
|
||||||
factory.registerFunction("emptyArrayDate", F { return new FunctionEmptyArrayDate; });
|
factory.registerFunction<FunctionEmptyArrayString>();
|
||||||
factory.registerFunction("emptyArrayDateTime", F { return new FunctionEmptyArrayDateTime; });
|
|
||||||
factory.registerFunction("emptyArrayString", F { return new FunctionEmptyArrayString; });
|
|
||||||
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,18 +6,14 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsCoding(FunctionFactory & factory)
|
void registerFunctionsCoding(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction *
|
factory.registerFunction<FunctionToStringCutToZero>();
|
||||||
|
factory.registerFunction<FunctionIPv6NumToString>();
|
||||||
factory.registerFunction("toStringCutToZero", F { return new FunctionToStringCutToZero; });
|
factory.registerFunction<FunctionIPv6StringToNum>();
|
||||||
factory.registerFunction("IPv6NumToString", F { return new FunctionIPv6NumToString; });
|
factory.registerFunction<FunctionIPv4NumToString>();
|
||||||
factory.registerFunction("IPv6StringToNum", F { return new FunctionIPv6StringToNum; });
|
factory.registerFunction<FunctionIPv4StringToNum>();
|
||||||
factory.registerFunction("IPv4NumToString", F { return new FunctionIPv4NumToString; });
|
factory.registerFunction<FunctionHex>();
|
||||||
factory.registerFunction("IPv4StringToNum", F { return new FunctionIPv4StringToNum; });
|
factory.registerFunction<FunctionUnhex>();
|
||||||
factory.registerFunction("hex", F { return new FunctionHex; });
|
factory.registerFunction<FunctionBitmaskToArray>();
|
||||||
factory.registerFunction("unhex", F { return new FunctionUnhex; });
|
|
||||||
factory.registerFunction("bitmaskToArray", F { return new FunctionBitmaskToArray; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,16 +6,12 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsComparison(FunctionFactory & factory)
|
void registerFunctionsComparison(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionEquals>();
|
||||||
|
factory.registerFunction<FunctionNotEquals>();
|
||||||
factory.registerFunction("equals", F { return new FunctionEquals; });
|
factory.registerFunction<FunctionLess>();
|
||||||
factory.registerFunction("notEquals", F { return new FunctionNotEquals; });
|
factory.registerFunction<FunctionGreater>();
|
||||||
factory.registerFunction("less", F { return new FunctionLess; });
|
factory.registerFunction<FunctionLessOrEquals>();
|
||||||
factory.registerFunction("greater", F { return new FunctionGreater; });
|
factory.registerFunction<FunctionGreaterOrEquals>();
|
||||||
factory.registerFunction("lessOrEquals", F { return new FunctionLessOrEquals; });
|
|
||||||
factory.registerFunction("greaterOrEquals", F { return new FunctionGreaterOrEquals; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,11 +6,7 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsConditional(FunctionFactory & factory)
|
void registerFunctionsConditional(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionIf>();
|
||||||
|
|
||||||
factory.registerFunction("if", F { return new FunctionIf; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,24 +6,20 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsConversion(FunctionFactory & factory)
|
void registerFunctionsConversion(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionToUInt8>();
|
||||||
|
factory.registerFunction<FunctionToUInt16>();
|
||||||
factory.registerFunction("toUInt8", F { return new FunctionToUInt8; });
|
factory.registerFunction<FunctionToUInt32>();
|
||||||
factory.registerFunction("toUInt16", F { return new FunctionToUInt16; });
|
factory.registerFunction<FunctionToUInt64>();
|
||||||
factory.registerFunction("toUInt32", F { return new FunctionToUInt32; });
|
factory.registerFunction<FunctionToInt8>();
|
||||||
factory.registerFunction("toUInt64", F { return new FunctionToUInt64; });
|
factory.registerFunction<FunctionToInt16>();
|
||||||
factory.registerFunction("toInt8", F { return new FunctionToInt8; });
|
factory.registerFunction<FunctionToInt32>();
|
||||||
factory.registerFunction("toInt16", F { return new FunctionToInt16; });
|
factory.registerFunction<FunctionToInt64>();
|
||||||
factory.registerFunction("toInt32", F { return new FunctionToInt32; });
|
factory.registerFunction<FunctionToFloat32>();
|
||||||
factory.registerFunction("toInt64", F { return new FunctionToInt64; });
|
factory.registerFunction<FunctionToFloat64>();
|
||||||
factory.registerFunction("toFloat32", F { return new FunctionToFloat32; });
|
factory.registerFunction<FunctionToDate>();
|
||||||
factory.registerFunction("toFloat64", F { return new FunctionToFloat64; });
|
factory.registerFunction<FunctionToDateTime>();
|
||||||
factory.registerFunction("toDate", F { return new FunctionToDate; });
|
factory.registerFunction<FunctionToString>();
|
||||||
factory.registerFunction("toDateTime", F { return new FunctionToDateTime; });
|
factory.registerFunction<FunctionToFixedString>();
|
||||||
factory.registerFunction("toString", F { return new FunctionToString; });
|
|
||||||
factory.registerFunction("toFixedString", F { return new FunctionToFixedString; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,36 +6,32 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsDateTime(FunctionFactory & factory)
|
void registerFunctionsDateTime(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionToYear>();
|
||||||
|
factory.registerFunction<FunctionToMonth>();
|
||||||
factory.registerFunction("toYear", F { return new FunctionToYear; });
|
factory.registerFunction<FunctionToDayOfMonth>();
|
||||||
factory.registerFunction("toMonth", F { return new FunctionToMonth; });
|
factory.registerFunction<FunctionToDayOfWeek>();
|
||||||
factory.registerFunction("toDayOfMonth", F { return new FunctionToDayOfMonth; });
|
factory.registerFunction<FunctionToHour>();
|
||||||
factory.registerFunction("toDayOfWeek", F { return new FunctionToDayOfWeek; });
|
factory.registerFunction<FunctionToMinute>();
|
||||||
factory.registerFunction("toHour", F { return new FunctionToHour; });
|
factory.registerFunction<FunctionToSecond>();
|
||||||
factory.registerFunction("toMinute", F { return new FunctionToMinute; });
|
factory.registerFunction<FunctionToMonday>();
|
||||||
factory.registerFunction("toSecond", F { return new FunctionToSecond; });
|
factory.registerFunction<FunctionToStartOfMonth>();
|
||||||
factory.registerFunction("toMonday", F { return new FunctionToMonday; });
|
factory.registerFunction<FunctionToStartOfQuarter>();
|
||||||
factory.registerFunction("toStartOfMonth", F { return new FunctionToStartOfMonth; });
|
factory.registerFunction<FunctionToStartOfYear>();
|
||||||
factory.registerFunction("toStartOfQuarter", F { return new FunctionToStartOfQuarter; });
|
factory.registerFunction<FunctionToStartOfMinute>();
|
||||||
factory.registerFunction("toStartOfYear", F { return new FunctionToStartOfYear; });
|
factory.registerFunction<FunctionToStartOfHour>();
|
||||||
factory.registerFunction("toStartOfMinute", F { return new FunctionToStartOfMinute; });
|
factory.registerFunction<FunctionToRelativeYearNum>();
|
||||||
factory.registerFunction("toStartOfHour", F { return new FunctionToStartOfHour; });
|
factory.registerFunction<FunctionToRelativeMonthNum>();
|
||||||
factory.registerFunction("toRelativeYearNum", F { return new FunctionToRelativeYearNum; });
|
factory.registerFunction<FunctionToRelativeWeekNum>();
|
||||||
factory.registerFunction("toRelativeMonthNum", F { return new FunctionToRelativeMonthNum; });
|
factory.registerFunction<FunctionToRelativeDayNum>();
|
||||||
factory.registerFunction("toRelativeWeekNum", F { return new FunctionToRelativeWeekNum; });
|
factory.registerFunction<FunctionToRelativeHourNum>();
|
||||||
factory.registerFunction("toRelativeDayNum", F { return new FunctionToRelativeDayNum; });
|
factory.registerFunction<FunctionToRelativeMinuteNum>();
|
||||||
factory.registerFunction("toRelativeHourNum", F { return new FunctionToRelativeHourNum; });
|
factory.registerFunction<FunctionToRelativeSecondNum>();
|
||||||
factory.registerFunction("toRelativeMinuteNum", F { return new FunctionToRelativeMinuteNum; });
|
factory.registerFunction<FunctionToTime>();
|
||||||
factory.registerFunction("toRelativeSecondNum", F { return new FunctionToRelativeSecondNum; });
|
factory.registerFunction<FunctionNow>();
|
||||||
factory.registerFunction("toTime", F { return new FunctionToTime; });
|
factory.registerFunction<FunctionToday>();
|
||||||
factory.registerFunction("now", F { return new FunctionNow; });
|
factory.registerFunction<FunctionYesterday>();
|
||||||
factory.registerFunction("today", F { return new FunctionToday; });
|
factory.registerFunction<FunctionTimeSlot>();
|
||||||
factory.registerFunction("yesterday", F { return new FunctionYesterday; });
|
factory.registerFunction<FunctionTimeSlots>();
|
||||||
factory.registerFunction("timeSlot", F { return new FunctionTimeSlot; });
|
|
||||||
factory.registerFunction("timeSlots", F { return new FunctionTimeSlots; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,46 +6,24 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsDictionaries(FunctionFactory & factory)
|
void registerFunctionsDictionaries(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionRegionToCity>();
|
||||||
|
factory.registerFunction<FunctionRegionToArea>();
|
||||||
factory.registerFunction("regionToCity",
|
factory.registerFunction<FunctionRegionToDistrict>();
|
||||||
F { return new FunctionRegionToCity(context.getDictionaries().getRegionsHierarchies()); });
|
factory.registerFunction<FunctionRegionToCountry>();
|
||||||
factory.registerFunction("regionToArea",
|
factory.registerFunction<FunctionRegionToContinent>();
|
||||||
F { return new FunctionRegionToArea(context.getDictionaries().getRegionsHierarchies()); });
|
factory.registerFunction<FunctionOSToRoot>();
|
||||||
factory.registerFunction("regionToDistrict",
|
factory.registerFunction<FunctionSEToRoot>();
|
||||||
F { return new FunctionRegionToDistrict(context.getDictionaries().getRegionsHierarchies()); });
|
factory.registerFunction<FunctionCategoryToRoot>();
|
||||||
factory.registerFunction("regionToCountry",
|
factory.registerFunction<FunctionCategoryToSecondLevel>();
|
||||||
F { return new FunctionRegionToCountry(context.getDictionaries().getRegionsHierarchies()); });
|
factory.registerFunction<FunctionRegionIn>();
|
||||||
factory.registerFunction("regionToContinent",
|
factory.registerFunction<FunctionOSIn>();
|
||||||
F { return new FunctionRegionToContinent(context.getDictionaries().getRegionsHierarchies()); });
|
factory.registerFunction<FunctionSEIn>();
|
||||||
factory.registerFunction("OSToRoot",
|
factory.registerFunction<FunctionCategoryIn>();
|
||||||
F { return new FunctionOSToRoot(context.getDictionaries().getTechDataHierarchy()); });
|
factory.registerFunction<FunctionRegionHierarchy>();
|
||||||
factory.registerFunction("SEToRoot",
|
factory.registerFunction<FunctionOSHierarchy>();
|
||||||
F { return new FunctionSEToRoot(context.getDictionaries().getTechDataHierarchy()); });
|
factory.registerFunction<FunctionSEHierarchy>();
|
||||||
factory.registerFunction("categoryToRoot",
|
factory.registerFunction<FunctionCategoryHierarchy>();
|
||||||
F { return new FunctionCategoryToRoot(context.getDictionaries().getCategoriesHierarchy()); });
|
factory.registerFunction<FunctionRegionToName>();
|
||||||
factory.registerFunction("categoryToSecondLevel",
|
|
||||||
F { return new FunctionCategoryToSecondLevel(context.getDictionaries().getCategoriesHierarchy()); });
|
|
||||||
factory.registerFunction("regionIn",
|
|
||||||
F { return new FunctionRegionIn(context.getDictionaries().getRegionsHierarchies()); });
|
|
||||||
factory.registerFunction("OSIn",
|
|
||||||
F { return new FunctionOSIn(context.getDictionaries().getTechDataHierarchy()); });
|
|
||||||
factory.registerFunction("SEIn",
|
|
||||||
F { return new FunctionSEIn(context.getDictionaries().getTechDataHierarchy()); });
|
|
||||||
factory.registerFunction("categoryIn",
|
|
||||||
F { return new FunctionCategoryIn(context.getDictionaries().getCategoriesHierarchy()); });
|
|
||||||
factory.registerFunction("regionHierarchy",
|
|
||||||
F { return new FunctionRegionHierarchy(context.getDictionaries().getRegionsHierarchies()); });
|
|
||||||
factory.registerFunction("OSHierarchy",
|
|
||||||
F { return new FunctionOSHierarchy(context.getDictionaries().getTechDataHierarchy()); });
|
|
||||||
factory.registerFunction("SEHierarchy",
|
|
||||||
F { return new FunctionSEHierarchy(context.getDictionaries().getTechDataHierarchy()); });
|
|
||||||
factory.registerFunction("categoryHierarchy",
|
|
||||||
F { return new FunctionCategoryHierarchy(context.getDictionaries().getCategoriesHierarchy()); });
|
|
||||||
factory.registerFunction("regionToName",
|
|
||||||
F { return new FunctionRegionToName(context.getDictionaries().getRegionsNames()); });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,11 +6,7 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsFormatting(FunctionFactory & factory)
|
void registerFunctionsFormatting(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionBitmaskToList>();
|
||||||
|
|
||||||
factory.registerFunction("bitmaskToList", F { return new FunctionBitmaskToList; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -7,20 +7,16 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsHashing(FunctionFactory & factory)
|
void registerFunctionsHashing(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction *
|
factory.registerFunction<FunctionHalfMD5>();
|
||||||
|
factory.registerFunction<FunctionMD5>();
|
||||||
factory.registerFunction("halfMD5", F { return new FunctionHalfMD5; });
|
factory.registerFunction<FunctionSHA1>();
|
||||||
factory.registerFunction("MD5", F { return new FunctionMD5; });
|
factory.registerFunction<FunctionSHA224>();
|
||||||
factory.registerFunction("SHA1", F { return new FunctionSHA1; });
|
factory.registerFunction<FunctionSHA256>();
|
||||||
factory.registerFunction("SHA224", F { return new FunctionSHA224; });
|
factory.registerFunction<FunctionSipHash64>();
|
||||||
factory.registerFunction("SHA256", F { return new FunctionSHA256; });
|
factory.registerFunction<FunctionSipHash128>();
|
||||||
factory.registerFunction("sipHash64", F { return new FunctionSipHash64; });
|
factory.registerFunction<FunctionCityHash64>();
|
||||||
factory.registerFunction("sipHash128", F { return new FunctionSipHash128; });
|
factory.registerFunction<FunctionIntHash32>();
|
||||||
factory.registerFunction("cityHash64", F { return new FunctionCityHash64; });
|
factory.registerFunction<FunctionIntHash64>();
|
||||||
factory.registerFunction("intHash32", F { return new FunctionIntHash32; });
|
|
||||||
factory.registerFunction("intHash64", F { return new FunctionIntHash64; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,18 +6,14 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsHigherOrder(FunctionFactory & factory)
|
void registerFunctionsHigherOrder(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction *
|
factory.registerFunction<FunctionArrayMap>();
|
||||||
|
factory.registerFunction<FunctionArrayFilter>();
|
||||||
factory.registerFunction("arrayMap", F { return new FunctionArrayMap; });
|
factory.registerFunction<FunctionArrayCount>();
|
||||||
factory.registerFunction("arrayFilter", F { return new FunctionArrayFilter; });
|
factory.registerFunction<FunctionArrayExists>();
|
||||||
factory.registerFunction("arrayCount", F { return new FunctionArrayCount; });
|
factory.registerFunction<FunctionArrayAll>();
|
||||||
factory.registerFunction("arrayExists", F { return new FunctionArrayExists; });
|
factory.registerFunction<FunctionArraySum>();
|
||||||
factory.registerFunction("arrayAll", F { return new FunctionArrayAll; });
|
factory.registerFunction<FunctionArrayFirst>();
|
||||||
factory.registerFunction("arraySum", F { return new FunctionArraySum; });
|
factory.registerFunction<FunctionArrayFirstIndex>();
|
||||||
factory.registerFunction("arrayFirst", F { return new FunctionArrayFirst; });
|
|
||||||
factory.registerFunction("arrayFirstIndex", F { return new FunctionArrayFirstIndex; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,14 +6,10 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsLogical(FunctionFactory & factory)
|
void registerFunctionsLogical(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionAnd>();
|
||||||
|
factory.registerFunction<FunctionOr>();
|
||||||
factory.registerFunction("and", F { return new FunctionAnd; });
|
factory.registerFunction<FunctionXor>();
|
||||||
factory.registerFunction("or", F { return new FunctionOr; });
|
factory.registerFunction<FunctionNot>();
|
||||||
factory.registerFunction("xor", F { return new FunctionXor; });
|
|
||||||
factory.registerFunction("not", F { return new FunctionNot; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -305,27 +305,23 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionCurrentDatabase>();
|
||||||
|
factory.registerFunction<FunctionHostName>();
|
||||||
|
factory.registerFunction<FunctionVisibleWidth>();
|
||||||
|
factory.registerFunction<FunctionToTypeName>();
|
||||||
|
factory.registerFunction<FunctionBlockSize>();
|
||||||
|
factory.registerFunction<FunctionSleep>();
|
||||||
|
factory.registerFunction<FunctionMaterialize>();
|
||||||
|
factory.registerFunction<FunctionIgnore>();
|
||||||
|
factory.registerFunction<FunctionArrayJoin>();
|
||||||
|
factory.registerFunction<FunctionBar>();
|
||||||
|
|
||||||
factory.registerFunction("currentDatabase", F { return new FunctionCurrentDatabase{context.getCurrentDatabase()}; });
|
factory.registerFunction<FunctionTuple>();
|
||||||
factory.registerFunction("hostName", F { return new FunctionHostName; });
|
factory.registerFunction<FunctionTupleElement>();
|
||||||
factory.registerFunction("visibleWidth", F { return new FunctionVisibleWidth; });
|
factory.registerFunction<FunctionIn<false, false>>();
|
||||||
factory.registerFunction("toTypeName", F { return new FunctionToTypeName; });
|
factory.registerFunction<FunctionIn<false, true>>();
|
||||||
factory.registerFunction("blockSize", F { return new FunctionBlockSize; });
|
factory.registerFunction<FunctionIn<true, false>>();
|
||||||
factory.registerFunction("sleep", F { return new FunctionSleep; });
|
factory.registerFunction<FunctionIn<true, true>>();
|
||||||
factory.registerFunction("materialize", F { return new FunctionMaterialize; });
|
|
||||||
factory.registerFunction("ignore", F { return new FunctionIgnore; });
|
|
||||||
factory.registerFunction("arrayJoin", F { return new FunctionArrayJoin; });
|
|
||||||
factory.registerFunction("bar", F { return new FunctionBar; });
|
|
||||||
|
|
||||||
factory.registerFunction("tuple", F { return new FunctionTuple; });
|
|
||||||
factory.registerFunction("tupleElement", F { return new FunctionTupleElement; });
|
|
||||||
factory.registerFunction("in", F { return new FunctionIn(false, false); });
|
|
||||||
factory.registerFunction("notIn", F { return new FunctionIn(true, false); });
|
|
||||||
factory.registerFunction("globalIn", F { return new FunctionIn(false, true); });
|
|
||||||
factory.registerFunction("globalNotIn", F { return new FunctionIn(true, true); });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,12 +6,8 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsRandom(FunctionFactory & factory)
|
void registerFunctionsRandom(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionRand>();
|
||||||
|
factory.registerFunction<FunctionRand64>();
|
||||||
factory.registerFunction("rand", F { return new FunctionRand; });
|
|
||||||
factory.registerFunction("rand64", F { return new FunctionRand64; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,23 +6,19 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsReinterpret(FunctionFactory & factory)
|
void registerFunctionsReinterpret(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionReinterpretAsUInt8>();
|
||||||
|
factory.registerFunction<FunctionReinterpretAsUInt16>();
|
||||||
factory.registerFunction("reinterpretAsUInt8", F { return new FunctionReinterpretAsUInt8; });
|
factory.registerFunction<FunctionReinterpretAsUInt32>();
|
||||||
factory.registerFunction("reinterpretAsUInt16", F { return new FunctionReinterpretAsUInt16; });
|
factory.registerFunction<FunctionReinterpretAsUInt64>();
|
||||||
factory.registerFunction("reinterpretAsUInt32", F { return new FunctionReinterpretAsUInt32; });
|
factory.registerFunction<FunctionReinterpretAsInt8>();
|
||||||
factory.registerFunction("reinterpretAsUInt64", F { return new FunctionReinterpretAsUInt64; });
|
factory.registerFunction<FunctionReinterpretAsInt16>();
|
||||||
factory.registerFunction("reinterpretAsInt8", F { return new FunctionReinterpretAsInt8; });
|
factory.registerFunction<FunctionReinterpretAsInt32>();
|
||||||
factory.registerFunction("reinterpretAsInt16", F { return new FunctionReinterpretAsInt16; });
|
factory.registerFunction<FunctionReinterpretAsInt64>();
|
||||||
factory.registerFunction("reinterpretAsInt32", F { return new FunctionReinterpretAsInt32; });
|
factory.registerFunction<FunctionReinterpretAsFloat32>();
|
||||||
factory.registerFunction("reinterpretAsInt64", F { return new FunctionReinterpretAsInt64; });
|
factory.registerFunction<FunctionReinterpretAsFloat64>();
|
||||||
factory.registerFunction("reinterpretAsFloat32", F { return new FunctionReinterpretAsFloat32; });
|
factory.registerFunction<FunctionReinterpretAsDate>();
|
||||||
factory.registerFunction("reinterpretAsFloat64", F { return new FunctionReinterpretAsFloat64; });
|
factory.registerFunction<FunctionReinterpretAsDateTime>();
|
||||||
factory.registerFunction("reinterpretAsDate", F { return new FunctionReinterpretAsDate; });
|
factory.registerFunction<FunctionReinterpretAsString>();
|
||||||
factory.registerFunction("reinterpretAsDateTime", F { return new FunctionReinterpretAsDateTime; });
|
|
||||||
factory.registerFunction("reinterpretAsString", F { return new FunctionReinterpretAsString; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,13 +6,9 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsRound(FunctionFactory & factory)
|
void registerFunctionsRound(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionRoundToExp2>();
|
||||||
|
factory.registerFunction<FunctionRoundDuration>();
|
||||||
factory.registerFunction("roundToExp2", F { return new FunctionRoundToExp2; });
|
factory.registerFunction<FunctionRoundAge>();
|
||||||
factory.registerFunction("roundDuration", F { return new FunctionRoundDuration; });
|
|
||||||
factory.registerFunction("roundAge", F { return new FunctionRoundAge; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,23 +6,19 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsString(FunctionFactory & factory)
|
void registerFunctionsString(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionEmpty>();
|
||||||
|
factory.registerFunction<FunctionNotEmpty>();
|
||||||
factory.registerFunction("empty", F { return new FunctionEmpty; });
|
factory.registerFunction<FunctionLength>();
|
||||||
factory.registerFunction("notEmpty", F { return new FunctionNotEmpty; });
|
factory.registerFunction<FunctionLengthUTF8>();
|
||||||
factory.registerFunction("length", F { return new FunctionLength; });
|
factory.registerFunction<FunctionLower>();
|
||||||
factory.registerFunction("lengthUTF8", F { return new FunctionLengthUTF8; });
|
factory.registerFunction<FunctionUpper>();
|
||||||
factory.registerFunction("lower", F { return new FunctionLower; });
|
factory.registerFunction<FunctionLowerUTF8>();
|
||||||
factory.registerFunction("upper", F { return new FunctionUpper; });
|
factory.registerFunction<FunctionUpperUTF8>();
|
||||||
factory.registerFunction("lowerUTF8", F { return new FunctionLowerUTF8; });
|
factory.registerFunction<FunctionReverse>();
|
||||||
factory.registerFunction("upperUTF8", F { return new FunctionUpperUTF8; });
|
factory.registerFunction<FunctionReverseUTF8>();
|
||||||
factory.registerFunction("reverse", F { return new FunctionReverse; });
|
factory.registerFunction<FunctionConcat>();
|
||||||
factory.registerFunction("reverseUTF8", F { return new FunctionReverseUTF8; });
|
factory.registerFunction<FunctionSubstring>();
|
||||||
factory.registerFunction("concat", F { return new FunctionConcat; });
|
factory.registerFunction<FunctionSubstringUTF8>();
|
||||||
factory.registerFunction("substring", F { return new FunctionSubstring; });
|
|
||||||
factory.registerFunction("substringUTF8", F { return new FunctionSubstringUTF8; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,14 +6,10 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsStringArray(FunctionFactory & factory)
|
void registerFunctionsStringArray(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionExtractAll>();
|
||||||
|
factory.registerFunction<FunctionAlphaTokens>();
|
||||||
factory.registerFunction("extractAll", F { return new FunctionExtractAll; });
|
factory.registerFunction<FunctionSplitByChar>();
|
||||||
factory.registerFunction("alphaTokens", F { return new FunctionAlphaTokens; });
|
factory.registerFunction<FunctionSplitByString>();
|
||||||
factory.registerFunction("splitByChar", F { return new FunctionSplitByChar; });
|
|
||||||
factory.registerFunction("splitByString", F { return new FunctionSplitByString; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,20 +6,16 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsStringSearch(FunctionFactory & factory)
|
void registerFunctionsStringSearch(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionReplaceOne>();
|
||||||
|
factory.registerFunction<FunctionReplaceAll>();
|
||||||
factory.registerFunction("replaceOne", F { return new FunctionReplaceOne; });
|
factory.registerFunction<FunctionReplaceRegexpOne>();
|
||||||
factory.registerFunction("replaceAll", F { return new FunctionReplaceAll; });
|
factory.registerFunction<FunctionReplaceRegexpAll>();
|
||||||
factory.registerFunction("replaceRegexpOne", F { return new FunctionReplaceRegexpOne; });
|
factory.registerFunction<FunctionPosition>();
|
||||||
factory.registerFunction("replaceRegexpAll", F { return new FunctionReplaceRegexpAll; });
|
factory.registerFunction<FunctionPositionUTF8>();
|
||||||
factory.registerFunction("position", F { return new FunctionPosition; });
|
factory.registerFunction<FunctionMatch>();
|
||||||
factory.registerFunction("positionUTF8", F { return new FunctionPositionUTF8; });
|
factory.registerFunction<FunctionLike>();
|
||||||
factory.registerFunction("match", F { return new FunctionMatch; });
|
factory.registerFunction<FunctionNotLike>();
|
||||||
factory.registerFunction("like", F { return new FunctionLike; });
|
factory.registerFunction<FunctionExtract>();
|
||||||
factory.registerFunction("notLike", F { return new FunctionNotLike; });
|
|
||||||
factory.registerFunction("extract", F { return new FunctionExtract; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,30 +6,26 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsURL(FunctionFactory & factory)
|
void registerFunctionsURL(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionProtocol>();
|
||||||
|
factory.registerFunction<FunctionDomain>();
|
||||||
factory.registerFunction("protocol", F { return new FunctionProtocol; });
|
factory.registerFunction<FunctionDomainWithoutWWW>();
|
||||||
factory.registerFunction("domain", F { return new FunctionDomain; });
|
factory.registerFunction<FunctionFirstSignificantSubdomain>();
|
||||||
factory.registerFunction("domainWithoutWWW", F { return new FunctionDomainWithoutWWW; });
|
factory.registerFunction<FunctionTopLevelDomain>();
|
||||||
factory.registerFunction("firstSignificantSubdomain", F { return new FunctionFirstSignificantSubdomain; });
|
factory.registerFunction<FunctionPath>();
|
||||||
factory.registerFunction("topLevelDomain", F { return new FunctionTopLevelDomain; });
|
factory.registerFunction<FunctionQueryString>();
|
||||||
factory.registerFunction("path", F { return new FunctionPath; });
|
factory.registerFunction<FunctionFragment>();
|
||||||
factory.registerFunction("queryString", F { return new FunctionQueryString; });
|
factory.registerFunction<FunctionQueryStringAndFragment>();
|
||||||
factory.registerFunction("fragment", F { return new FunctionFragment; });
|
factory.registerFunction<FunctionExtractURLParameter>();
|
||||||
factory.registerFunction("queryStringAndFragment", F { return new FunctionQueryStringAndFragment; });
|
factory.registerFunction<FunctionExtractURLParameters>();
|
||||||
factory.registerFunction("extractURLParameter", F { return new FunctionExtractURLParameter; });
|
factory.registerFunction<FunctionExtractURLParameterNames>();
|
||||||
factory.registerFunction("extractURLParameters", F { return new FunctionExtractURLParameters; });
|
factory.registerFunction<FunctionURLHierarchy>();
|
||||||
factory.registerFunction("extractURLParameterNames", F { return new FunctionExtractURLParameterNames; });
|
factory.registerFunction<FunctionURLPathHierarchy>();
|
||||||
factory.registerFunction("URLHierarchy", F { return new FunctionURLHierarchy; });
|
factory.registerFunction<FunctionCutToFirstSignificantSubdomain>();
|
||||||
factory.registerFunction("URLPathHierarchy", F { return new FunctionURLPathHierarchy; });
|
factory.registerFunction<FunctionCutWWW>();
|
||||||
factory.registerFunction("cutToFirstSignificantSubdomain", F { return new FunctionCutToFirstSignificantSubdomain; });
|
factory.registerFunction<FunctionCutQueryString>();
|
||||||
factory.registerFunction("cutWWW", F { return new FunctionCutWWW; });
|
factory.registerFunction<FunctionCutFragment>();
|
||||||
factory.registerFunction("cutQueryString", F { return new FunctionCutQueryString; });
|
factory.registerFunction<FunctionCutQueryStringAndFragment>();
|
||||||
factory.registerFunction("cutFragment", F { return new FunctionCutFragment; });
|
factory.registerFunction<FunctionCutURLParameter>();
|
||||||
factory.registerFunction("cutQueryStringAndFragment", F { return new FunctionCutQueryStringAndFragment; });
|
|
||||||
factory.registerFunction("cutURLParameter", F { return new FunctionCutURLParameter; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -6,17 +6,13 @@ namespace DB
|
|||||||
|
|
||||||
void registerFunctionsVisitParam(FunctionFactory & factory)
|
void registerFunctionsVisitParam(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
#define F [](const Context & context) -> IFunction*
|
factory.registerFunction<FunctionVisitParamHas>();
|
||||||
|
factory.registerFunction<FunctionVisitParamExtractUInt>();
|
||||||
factory.registerFunction("visitParamHas", F { return new FunctionVisitParamHas; });
|
factory.registerFunction<FunctionVisitParamExtractInt>();
|
||||||
factory.registerFunction("visitParamExtractUInt", F { return new FunctionVisitParamExtractUInt; });
|
factory.registerFunction<FunctionVisitParamExtractFloat>();
|
||||||
factory.registerFunction("visitParamExtractInt", F { return new FunctionVisitParamExtractInt; });
|
factory.registerFunction<FunctionVisitParamExtractBool>();
|
||||||
factory.registerFunction("visitParamExtractFloat", F { return new FunctionVisitParamExtractFloat; });
|
factory.registerFunction<FunctionVisitParamExtractRaw>();
|
||||||
factory.registerFunction("visitParamExtractBool", F { return new FunctionVisitParamExtractBool; });
|
factory.registerFunction<FunctionVisitParamExtractString>();
|
||||||
factory.registerFunction("visitParamExtractRaw", F { return new FunctionVisitParamExtractRaw; });
|
|
||||||
factory.registerFunction("visitParamExtractString", F { return new FunctionVisitParamExtractString; });
|
|
||||||
|
|
||||||
#undef F
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -63,8 +63,10 @@ void ExpressionAnalyzer::init()
|
|||||||
{
|
{
|
||||||
select_query = typeid_cast<ASTSelectQuery *>(&*ast);
|
select_query = typeid_cast<ASTSelectQuery *>(&*ast);
|
||||||
|
|
||||||
|
addStorageAliases();
|
||||||
|
|
||||||
/// Создаёт словарь aliases: alias -> ASTPtr
|
/// Создаёт словарь aliases: alias -> ASTPtr
|
||||||
createAliasesDict(ast);
|
addASTAliases(ast);
|
||||||
|
|
||||||
/// Common subexpression elimination. Rewrite rules.
|
/// Common subexpression elimination. Rewrite rules.
|
||||||
normalizeTree();
|
normalizeTree();
|
||||||
@ -235,9 +237,19 @@ NamesAndTypesList::iterator ExpressionAnalyzer::findColumn(const String & name,
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void ExpressionAnalyzer::addStorageAliases()
|
||||||
|
{
|
||||||
|
if (!storage)
|
||||||
|
return;
|
||||||
|
|
||||||
|
for (const auto & alias : storage->alias_columns)
|
||||||
|
aliases[alias.name] = storage->column_defaults[alias.name].expression;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/// ignore_levels - алиасы в скольки верхних уровнях поддерева нужно игнорировать.
|
/// ignore_levels - алиасы в скольки верхних уровнях поддерева нужно игнорировать.
|
||||||
/// Например, при ignore_levels=1 ast не может быть занесен в словарь, но его дети могут.
|
/// Например, при ignore_levels=1 ast не может быть занесен в словарь, но его дети могут.
|
||||||
void ExpressionAnalyzer::createAliasesDict(ASTPtr & ast, int ignore_levels)
|
void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels)
|
||||||
{
|
{
|
||||||
ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(&*ast);
|
ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(&*ast);
|
||||||
|
|
||||||
@ -252,7 +264,7 @@ void ExpressionAnalyzer::createAliasesDict(ASTPtr & ast, int ignore_levels)
|
|||||||
new_ignore_levels = 2;
|
new_ignore_levels = 2;
|
||||||
|
|
||||||
if (!typeid_cast<ASTSelectQuery *>(&*child))
|
if (!typeid_cast<ASTSelectQuery *>(&*child))
|
||||||
createAliasesDict(child, new_ignore_levels);
|
addASTAliases(child, new_ignore_levels);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (ignore_levels > 0)
|
if (ignore_levels > 0)
|
||||||
@ -1689,11 +1701,26 @@ void ExpressionAnalyzer::collectUsedColumns()
|
|||||||
++it;
|
++it;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (NamesAndTypesList::iterator it = columns.begin(); it != columns.end();)
|
||||||
|
{
|
||||||
|
unknown_required_columns.erase(it->name);
|
||||||
|
|
||||||
|
if (!required.count(it->name))
|
||||||
|
{
|
||||||
|
required.erase(it->name);
|
||||||
|
columns.erase(it++);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
++it;
|
||||||
|
}
|
||||||
|
|
||||||
/// Возможно, среди неизвестных столбцов есть виртуальные. Удаляем их из списка неизвестных и добавляем
|
/// Возможно, среди неизвестных столбцов есть виртуальные. Удаляем их из списка неизвестных и добавляем
|
||||||
/// в columns list, чтобы при дальнейшей обработке запроса они воспринимались как настоящие.
|
/// в columns list, чтобы при дальнейшей обработке запроса они воспринимались как настоящие.
|
||||||
for (NameSet::iterator it = unknown_required_columns.begin(); it != unknown_required_columns.end();)
|
if (storage)
|
||||||
{
|
{
|
||||||
if (storage && storage->hasColumn(*it))
|
for (auto it = unknown_required_columns.begin(); it != unknown_required_columns.end();)
|
||||||
|
{
|
||||||
|
if (storage->hasColumn(*it))
|
||||||
{
|
{
|
||||||
columns.push_back(storage->getColumn(*it));
|
columns.push_back(storage->getColumn(*it));
|
||||||
unknown_required_columns.erase(it++);
|
unknown_required_columns.erase(it++);
|
||||||
@ -1702,6 +1729,7 @@ void ExpressionAnalyzer::collectUsedColumns()
|
|||||||
++it;
|
++it;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type)
|
void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type)
|
||||||
{
|
{
|
||||||
|
@ -28,15 +28,15 @@ InterpreterAlterQuery::InterpreterAlterQuery(ASTPtr query_ptr_, Context & contex
|
|||||||
|
|
||||||
void InterpreterAlterQuery::execute()
|
void InterpreterAlterQuery::execute()
|
||||||
{
|
{
|
||||||
ASTAlterQuery & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
||||||
String & table_name = alter.table;
|
const String & table_name = alter.table;
|
||||||
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
||||||
|
StoragePtr table = context.getTable(database_name, table_name);
|
||||||
|
|
||||||
AlterCommands alter_commands;
|
AlterCommands alter_commands;
|
||||||
PartitionCommands partition_commands;
|
PartitionCommands partition_commands;
|
||||||
parseAlter(alter.parameters, context.getDataTypeFactory(), alter_commands, partition_commands);
|
parseAlter(alter.parameters, context.getDataTypeFactory(), alter_commands, partition_commands);
|
||||||
|
|
||||||
StoragePtr table = context.getTable(database_name, table_name);
|
|
||||||
|
|
||||||
for (const PartitionCommand & command : partition_commands)
|
for (const PartitionCommand & command : partition_commands)
|
||||||
{
|
{
|
||||||
switch (command.type)
|
switch (command.type)
|
||||||
@ -62,7 +62,11 @@ void InterpreterAlterQuery::execute()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!alter_commands.empty())
|
if (alter_commands.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
alter_commands.validate(table.get(), context);
|
||||||
|
|
||||||
table->alter(alter_commands, database_name, table_name, context);
|
table->alter(alter_commands, database_name, table_name, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -77,12 +81,20 @@ void InterpreterAlterQuery::parseAlter(
|
|||||||
AlterCommand command;
|
AlterCommand command;
|
||||||
command.type = AlterCommand::ADD;
|
command.type = AlterCommand::ADD;
|
||||||
|
|
||||||
const ASTNameTypePair & ast_name_type = typeid_cast<const ASTNameTypePair &>(*params.name_type);
|
const auto & ast_col_decl = typeid_cast<const ASTColumnDeclaration &>(*params.col_decl);
|
||||||
StringRange type_range = ast_name_type.type->range;
|
|
||||||
String type_string = String(type_range.first, type_range.second - type_range.first);
|
|
||||||
|
|
||||||
command.column_name = ast_name_type.name;
|
command.column_name = ast_col_decl.name;
|
||||||
|
if (ast_col_decl.type)
|
||||||
|
{
|
||||||
|
StringRange type_range = ast_col_decl.type->range;
|
||||||
|
String type_string(type_range.first, type_range.second - type_range.first);
|
||||||
command.data_type = data_type_factory.get(type_string);
|
command.data_type = data_type_factory.get(type_string);
|
||||||
|
}
|
||||||
|
if (ast_col_decl.default_expression)
|
||||||
|
{
|
||||||
|
command.default_type = columnDefaultTypeFromString(ast_col_decl.default_specifier);
|
||||||
|
command.default_expression = ast_col_decl.default_expression;
|
||||||
|
}
|
||||||
|
|
||||||
if (params.column)
|
if (params.column)
|
||||||
command.after_column = typeid_cast<const ASTIdentifier &>(*params.column).name;
|
command.after_column = typeid_cast<const ASTIdentifier &>(*params.column).name;
|
||||||
@ -102,12 +114,21 @@ void InterpreterAlterQuery::parseAlter(
|
|||||||
AlterCommand command;
|
AlterCommand command;
|
||||||
command.type = AlterCommand::MODIFY;
|
command.type = AlterCommand::MODIFY;
|
||||||
|
|
||||||
const ASTNameTypePair & ast_name_type = typeid_cast<const ASTNameTypePair &>(*params.name_type);
|
const auto & ast_col_decl = typeid_cast<const ASTColumnDeclaration &>(*params.col_decl);
|
||||||
StringRange type_range = ast_name_type.type->range;
|
|
||||||
String type_string = String(type_range.first, type_range.second - type_range.first);
|
|
||||||
|
|
||||||
command.column_name = ast_name_type.name;
|
command.column_name = ast_col_decl.name;
|
||||||
|
if (ast_col_decl.type)
|
||||||
|
{
|
||||||
|
StringRange type_range = ast_col_decl.type->range;
|
||||||
|
String type_string(type_range.first, type_range.second - type_range.first);
|
||||||
command.data_type = data_type_factory.get(type_string);
|
command.data_type = data_type_factory.get(type_string);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ast_col_decl.default_expression)
|
||||||
|
{
|
||||||
|
command.default_type = columnDefaultTypeFromString(ast_col_decl.default_specifier);
|
||||||
|
command.default_expression = ast_col_decl.default_expression;
|
||||||
|
}
|
||||||
|
|
||||||
out_alter_commands.push_back(command);
|
out_alter_commands.push_back(command);
|
||||||
}
|
}
|
||||||
@ -137,7 +158,13 @@ void InterpreterAlterQuery::parseAlter(
|
|||||||
}
|
}
|
||||||
|
|
||||||
void InterpreterAlterQuery::updateMetadata(
|
void InterpreterAlterQuery::updateMetadata(
|
||||||
const String & database_name, const String & table_name, const NamesAndTypesList & columns, Context & context)
|
const String & database_name,
|
||||||
|
const String & table_name,
|
||||||
|
const NamesAndTypesList & columns,
|
||||||
|
const NamesAndTypesList & materialized_columns,
|
||||||
|
const NamesAndTypesList & alias_columns,
|
||||||
|
const ColumnDefaults & column_defaults,
|
||||||
|
Context & context)
|
||||||
{
|
{
|
||||||
String path = context.getPath();
|
String path = context.getPath();
|
||||||
|
|
||||||
@ -172,7 +199,7 @@ void InterpreterAlterQuery::updateMetadata(
|
|||||||
|
|
||||||
ASTCreateQuery & attach = typeid_cast<ASTCreateQuery &>(*ast);
|
ASTCreateQuery & attach = typeid_cast<ASTCreateQuery &>(*ast);
|
||||||
|
|
||||||
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns, materialized_columns, alias_columns, column_defaults);
|
||||||
*std::find(attach.children.begin(), attach.children.end(), attach.columns) = new_columns;
|
*std::find(attach.children.begin(), attach.children.end(), attach.columns) = new_columns;
|
||||||
attach.columns = new_columns;
|
attach.columns = new_columns;
|
||||||
|
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
|
|
||||||
#include <DB/Parsers/ASTCreateQuery.h>
|
#include <DB/Parsers/ASTCreateQuery.h>
|
||||||
#include <DB/Parsers/ASTNameTypePair.h>
|
#include <DB/Parsers/ASTNameTypePair.h>
|
||||||
|
#include <DB/Parsers/ASTColumnDeclaration.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageLog.h>
|
#include <DB/Storages/StorageLog.h>
|
||||||
#include <DB/Storages/StorageSystemNumbers.h>
|
#include <DB/Storages/StorageSystemNumbers.h>
|
||||||
@ -20,6 +21,8 @@
|
|||||||
|
|
||||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||||
|
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataTypes/DataTypeNested.h>
|
#include <DB/DataTypes/DataTypeNested.h>
|
||||||
|
|
||||||
|
|
||||||
@ -87,6 +90,9 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
|||||||
StoragePtr res;
|
StoragePtr res;
|
||||||
String storage_name;
|
String storage_name;
|
||||||
NamesAndTypesListPtr columns = new NamesAndTypesList;
|
NamesAndTypesListPtr columns = new NamesAndTypesList;
|
||||||
|
NamesAndTypesList materialized_columns{};
|
||||||
|
NamesAndTypesList alias_columns{};
|
||||||
|
ColumnDefaults column_defaults{};
|
||||||
|
|
||||||
StoragePtr as_storage;
|
StoragePtr as_storage;
|
||||||
IStorage::TableStructureReadLockPtr as_storage_lock;
|
IStorage::TableStructureReadLockPtr as_storage_lock;
|
||||||
@ -115,10 +121,22 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
|||||||
/// Получаем список столбцов
|
/// Получаем список столбцов
|
||||||
if (create.columns)
|
if (create.columns)
|
||||||
{
|
{
|
||||||
columns = new NamesAndTypesList(parseColumns(create.columns, context.getDataTypeFactory()));
|
auto && columns_and_defaults = parseColumns(create.columns);
|
||||||
|
materialized_columns = removeAndReturnColumns(columns_and_defaults, ColumnDefaultType::Materialized);
|
||||||
|
alias_columns = removeAndReturnColumns(columns_and_defaults, ColumnDefaultType::Alias);
|
||||||
|
columns = new NamesAndTypesList{std::move(columns_and_defaults.first)};
|
||||||
|
column_defaults = std::move(columns_and_defaults.second);
|
||||||
|
|
||||||
|
if (columns->size() + materialized_columns.size() == 0)
|
||||||
|
throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED};
|
||||||
}
|
}
|
||||||
else if (!create.as_table.empty())
|
else if (!create.as_table.empty())
|
||||||
columns = new NamesAndTypesList(as_storage->getColumnsList());
|
{
|
||||||
|
columns = new NamesAndTypesList(as_storage->getColumnsListNonMaterialized());
|
||||||
|
materialized_columns = as_storage->materialized_columns;
|
||||||
|
alias_columns = as_storage->alias_columns;
|
||||||
|
column_defaults = as_storage->column_defaults;
|
||||||
|
}
|
||||||
else if (create.select)
|
else if (create.select)
|
||||||
{
|
{
|
||||||
columns = new NamesAndTypesList;
|
columns = new NamesAndTypesList;
|
||||||
@ -129,7 +147,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
|||||||
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
|
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
|
||||||
|
|
||||||
/// Даже если в запросе был список столбцов, на всякий случай приведем его к стандартному виду (развернем Nested).
|
/// Даже если в запросе был список столбцов, на всякий случай приведем его к стандартному виду (развернем Nested).
|
||||||
ASTPtr new_columns = formatColumns(*columns);
|
ASTPtr new_columns = formatColumns(*columns, materialized_columns, alias_columns, column_defaults);
|
||||||
if (create.columns)
|
if (create.columns)
|
||||||
{
|
{
|
||||||
auto it = std::find(create.children.begin(), create.children.end(), create.columns);
|
auto it = std::find(create.children.begin(), create.children.end(), create.columns);
|
||||||
@ -178,7 +196,8 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
|||||||
|
|
||||||
res = context.getStorageFactory().get(
|
res = context.getStorageFactory().get(
|
||||||
storage_name, data_path, table_name, database_name, context,
|
storage_name, data_path, table_name, database_name, context,
|
||||||
context.getGlobalContext(), query_ptr, columns, create.attach);
|
context.getGlobalContext(), query_ptr, columns,
|
||||||
|
materialized_columns, alias_columns, column_defaults, create.attach);
|
||||||
|
|
||||||
/// Проверка наличия метаданных таблицы на диске и создание метаданных
|
/// Проверка наличия метаданных таблицы на диске и создание метаданных
|
||||||
if (!assume_metadata_exists && !create.is_temporary)
|
if (!assume_metadata_exists && !create.is_temporary)
|
||||||
@ -233,47 +252,195 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
NamesAndTypesList InterpreterCreateQuery::parseColumns(ASTPtr expression_list, const DataTypeFactory & data_type_factory)
|
InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(ASTPtr expression_list)
|
||||||
{
|
{
|
||||||
NamesAndTypesList columns;
|
auto & column_list_ast = typeid_cast<ASTExpressionList &>(*expression_list);
|
||||||
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*expression_list);
|
|
||||||
for (const ASTPtr & ast : columns_list.children)
|
/// list of table columns in correct order
|
||||||
|
NamesAndTypesList columns{};
|
||||||
|
ColumnDefaults defaults{};
|
||||||
|
|
||||||
|
/// Columns requiring type-deduction or default_expression type-check
|
||||||
|
std::vector<std::pair<NameAndTypePair *, ASTColumnDeclaration *>> defaulted_columns{};
|
||||||
|
|
||||||
|
/** all default_expressions as a single expression list,
|
||||||
|
* mixed with conversion-columns for each explicitly specified type */
|
||||||
|
ASTPtr default_expr_list{new ASTExpressionList};
|
||||||
|
default_expr_list->children.reserve(column_list_ast.children.size());
|
||||||
|
|
||||||
|
for (auto & ast : column_list_ast.children)
|
||||||
{
|
{
|
||||||
const ASTNameTypePair & name_and_type_pair = typeid_cast<const ASTNameTypePair &>(*ast);
|
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
|
||||||
StringRange type_range = name_and_type_pair.type->range;
|
|
||||||
columns.push_back(NameAndTypePair(
|
if (col_decl.type)
|
||||||
name_and_type_pair.name,
|
{
|
||||||
data_type_factory.get(String(type_range.first, type_range.second - type_range.first))));
|
const auto & type_range = col_decl.type->range;
|
||||||
|
columns.emplace_back(col_decl.name,
|
||||||
|
context.getDataTypeFactory().get({ type_range.first, type_range.second }));
|
||||||
}
|
}
|
||||||
columns = *DataTypeNested::expandNestedColumns(columns);
|
else
|
||||||
return columns;
|
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||||
|
columns.emplace_back(col_decl.name, new DataTypeUInt8);
|
||||||
|
|
||||||
|
/// add column to postprocessing if there is a default_expression specified
|
||||||
|
if (col_decl.default_expression)
|
||||||
|
{
|
||||||
|
defaulted_columns.emplace_back(&columns.back(), &col_decl);
|
||||||
|
|
||||||
|
/** for columns with explicitly-specified type create two expressions:
|
||||||
|
* 1. default_expression aliased as column name with _tmp suffix
|
||||||
|
* 2. conversion of expression (1) to explicitly-specified type alias as column name */
|
||||||
|
if (col_decl.type)
|
||||||
|
{
|
||||||
|
const auto tmp_column_name = col_decl.name + "_tmp";
|
||||||
|
const auto & final_column_name = col_decl.name;
|
||||||
|
const auto conversion_function_name = "to" + columns.back().type->getName();
|
||||||
|
|
||||||
|
default_expr_list->children.emplace_back(setAlias(
|
||||||
|
makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}),
|
||||||
|
final_column_name));
|
||||||
|
|
||||||
|
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// set missing types and wrap default_expression's in a conversion-function if necessary
|
||||||
|
if (!defaulted_columns.empty())
|
||||||
|
{
|
||||||
|
const auto actions = ExpressionAnalyzer{default_expr_list, context, columns}.getActions(true);
|
||||||
|
const auto block = actions->getSampleBlock();
|
||||||
|
|
||||||
|
for (auto & column : defaulted_columns)
|
||||||
|
{
|
||||||
|
const auto name_and_type_ptr = column.first;
|
||||||
|
const auto col_decl_ptr = column.second;
|
||||||
|
|
||||||
|
if (col_decl_ptr->type)
|
||||||
|
{
|
||||||
|
const auto & tmp_column = block.getByName(col_decl_ptr->name + "_tmp");
|
||||||
|
|
||||||
|
/// type mismatch between explicitly specified and deduced type, add conversion
|
||||||
|
if (typeid(*name_and_type_ptr->type) != typeid(*tmp_column.type))
|
||||||
|
{
|
||||||
|
col_decl_ptr->default_expression = makeASTFunction(
|
||||||
|
"to" + name_and_type_ptr->type->getName(),
|
||||||
|
col_decl_ptr->default_expression);
|
||||||
|
|
||||||
|
col_decl_ptr->children.clear();
|
||||||
|
col_decl_ptr->children.push_back(col_decl_ptr->type);
|
||||||
|
col_decl_ptr->children.push_back(col_decl_ptr->default_expression);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
name_and_type_ptr->type = block.getByName(name_and_type_ptr->name).type;
|
||||||
|
|
||||||
|
defaults.emplace(col_decl_ptr->name, ColumnDefault{
|
||||||
|
columnDefaultTypeFromString(col_decl_ptr->default_specifier),
|
||||||
|
col_decl_ptr->default_expression
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return { *DataTypeNested::expandNestedColumns(columns), defaults };
|
||||||
|
}
|
||||||
|
|
||||||
|
NamesAndTypesList InterpreterCreateQuery::removeAndReturnColumns(ColumnsAndDefaults & columns_and_defaults,
|
||||||
|
const ColumnDefaultType type)
|
||||||
|
{
|
||||||
|
auto & columns = columns_and_defaults.first;
|
||||||
|
auto & defaults = columns_and_defaults.second;
|
||||||
|
|
||||||
|
NamesAndTypesList removed{};
|
||||||
|
|
||||||
|
for (auto it = std::begin(columns); it != std::end(columns);)
|
||||||
|
{
|
||||||
|
const auto jt = defaults.find(it->name);
|
||||||
|
if (jt != std::end(defaults) && jt->second.type == type)
|
||||||
|
{
|
||||||
|
removed.push_back(*it);
|
||||||
|
it = columns.erase(it);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
++it;
|
||||||
|
}
|
||||||
|
|
||||||
|
return removed;
|
||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
|
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
|
||||||
{
|
{
|
||||||
ASTPtr columns_list_ptr = new ASTExpressionList;
|
ASTPtr columns_list_ptr{new ASTExpressionList};
|
||||||
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
|
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
|
||||||
|
|
||||||
for (const NameAndTypePair & it : columns)
|
for (const auto & column : columns)
|
||||||
{
|
{
|
||||||
ASTPtr name_and_type_pair_ptr = new ASTNameTypePair;
|
const auto column_declaration = new ASTColumnDeclaration;
|
||||||
ASTNameTypePair & name_and_type_pair = typeid_cast<ASTNameTypePair &>(*name_and_type_pair_ptr);
|
ASTPtr column_declaration_ptr{column_declaration};
|
||||||
name_and_type_pair.name = it.name;
|
|
||||||
StringPtr type_name = new String(it.type->getName());
|
column_declaration->name = column.name;
|
||||||
|
|
||||||
|
StringPtr type_name{new String(column.type->getName())};
|
||||||
|
auto pos = type_name->data();
|
||||||
|
const auto end = pos + type_name->size();
|
||||||
|
|
||||||
ParserIdentifierWithOptionalParameters storage_p;
|
ParserIdentifierWithOptionalParameters storage_p;
|
||||||
Expected expected = "";
|
Expected expected{""};
|
||||||
const char * pos = type_name->data();
|
if (!storage_p.parse(pos, end, column_declaration->type, expected))
|
||||||
const char * end = pos + type_name->size();
|
|
||||||
|
|
||||||
if (!storage_p.parse(pos, end, name_and_type_pair.type, expected))
|
|
||||||
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
|
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
|
||||||
|
|
||||||
name_and_type_pair.type->query_string = type_name;
|
column_declaration->type->query_string = type_name;
|
||||||
columns_list.children.push_back(name_and_type_pair_ptr);
|
columns_list.children.push_back(column_declaration_ptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
return columns_list_ptr;
|
return columns_list_ptr;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
|
||||||
|
const NamesAndTypesList & materialized_columns,
|
||||||
|
const NamesAndTypesList & alias_columns,
|
||||||
|
const ColumnDefaults & column_defaults)
|
||||||
|
{
|
||||||
|
columns.insert(std::end(columns), std::begin(materialized_columns), std::end(materialized_columns));
|
||||||
|
columns.insert(std::end(columns), std::begin(alias_columns), std::end(alias_columns));
|
||||||
|
|
||||||
|
ASTPtr columns_list_ptr{new ASTExpressionList};
|
||||||
|
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
|
||||||
|
|
||||||
|
for (const auto & column : columns)
|
||||||
|
{
|
||||||
|
const auto column_declaration = new ASTColumnDeclaration;
|
||||||
|
ASTPtr column_declaration_ptr{column_declaration};
|
||||||
|
|
||||||
|
column_declaration->name = column.name;
|
||||||
|
|
||||||
|
StringPtr type_name{new String(column.type->getName())};
|
||||||
|
auto pos = type_name->data();
|
||||||
|
const auto end = pos + type_name->size();
|
||||||
|
|
||||||
|
ParserIdentifierWithOptionalParameters storage_p;
|
||||||
|
Expected expected{""};
|
||||||
|
if (!storage_p.parse(pos, end, column_declaration->type, expected))
|
||||||
|
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
|
||||||
|
|
||||||
|
column_declaration->type->query_string = type_name;
|
||||||
|
|
||||||
|
const auto it = column_defaults.find(column.name);
|
||||||
|
if (it != std::end(column_defaults))
|
||||||
|
{
|
||||||
|
column_declaration->default_specifier = toString(it->second.type);
|
||||||
|
column_declaration->default_expression = it->second.expression->clone();
|
||||||
|
}
|
||||||
|
|
||||||
|
columns_list.children.push_back(column_declaration_ptr);
|
||||||
|
}
|
||||||
|
|
||||||
|
return columns_list_ptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user