diff --git a/dbms/include/DB/Common/CompactArray.h b/dbms/include/DB/Common/CompactArray.h new file mode 100644 index 00000000000..a0f8b5e41a3 --- /dev/null +++ b/dbms/include/DB/Common/CompactArray.h @@ -0,0 +1,254 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/** Компактный массив для хранения данных, размер L, в битах, которых составляет + * меньше одного байта. Вместо того, чтобы хранить каждое значение в отдельный + * байт, что приводит к растрате 37.5% пространства для L=5, CompactArray хранит + * смежные L-битные значения в массиве байтов, т.е. фактически CompactArray + * симулирует массив L-битных значений. + */ +template +class __attribute__ ((packed)) CompactArray final +{ +public: + class Reader; + class Locus; + +public: + CompactArray() = default; + + UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const + { + Locus locus(bucket_index); + + if (locus.index_l == locus.index_r) + return locus.read(bitset[locus.index_l]); + else + return locus.read(bitset[locus.index_l], bitset[locus.index_r]); + } + + Locus ALWAYS_INLINE operator[](BucketIndex bucket_index) + { + Locus locus(bucket_index); + + locus.content_l = &bitset[locus.index_l]; + + if (locus.index_l == locus.index_r) + locus.content_r = locus.content_l; + else + locus.content_r = &bitset[locus.index_r]; + + return locus; + } + + void readText(ReadBuffer & in) + { + for (size_t i = 0; i < BITSET_SIZE; ++i) + { + if (i != 0) + assertString(",", in); + readIntText(bitset[i], in); + } + } + + void writeText(WriteBuffer & out) const + { + for (size_t i = 0; i < BITSET_SIZE; ++i) + { + if (i != 0) + writeCString(",", out); + writeIntText(bitset[i], out); + } + } + +private: + /// число байт в битсете + static constexpr size_t BITSET_SIZE = (static_cast(bucket_count) * content_width + 7) / 8; + UInt8 bitset[BITSET_SIZE] = { 0 }; +}; + +/** Класс для последовательного чтения ячеек из компактного массива на диске. + */ +template +class CompactArray::Reader final +{ +public: + Reader(ReadBuffer & in_) + : in(in_) + { + } + + Reader(const Reader &) = delete; + Reader & operator=(const Reader &) = delete; + + bool next() + { + if (current_bucket_index == bucket_count) + { + is_eof = true; + return false; + } + + locus.init(current_bucket_index); + + if (current_bucket_index == 0) + { + in.readStrict(reinterpret_cast(&value_l), 1); + ++read_count; + } + else + value_l = value_r; + + if (locus.index_l != locus.index_r) + { + if (read_count == BITSET_SIZE) + fits_in_byte = true; + else + { + fits_in_byte = false; + in.readStrict(reinterpret_cast(&value_r), 1); + ++read_count; + } + } + else + { + fits_in_byte = true; + value_r = value_l; + } + + ++current_bucket_index; + + return true; + } + + /** Вернуть текущий номер ячейки и соответствующее содержание. + */ + inline std::pair get() const + { + if ((current_bucket_index == 0) || is_eof) + throw Exception("No available data.", ErrorCodes::NO_AVAILABLE_DATA); + + if (fits_in_byte) + return std::make_pair(current_bucket_index - 1, locus.read(value_l)); + else + return std::make_pair(current_bucket_index - 1, locus.read(value_l, value_r)); + } + +private: + ReadBuffer & in; + /// Физическое расположение текущей ячейки. + Locus locus; + /// Текущая позиция в файле в виде номера ячейки. + BucketIndex current_bucket_index = 0; + /// Количество прочитанных байтов. + size_t read_count = 0; + /// Содержание в текущей позиции. + UInt8 value_l; + UInt8 value_r; + /// + bool is_eof = false; + /// Влезает ли ячейка полностью в один байт? + bool fits_in_byte; +}; + +/** Структура Locus содержит необходимую информацию, чтобы найти для каждой ячейки + * соответствующие байт и смещение, в битах, от начала ячейки. Поскольку в общем + * случае размер одного байта не делится на размер одной ячейки, возможны случаи, + * когда одна ячейка перекрывает два байта. Поэтому структура Locus содержит две + * пары (индекс, смещение). + */ +template +class CompactArray::Locus final +{ + friend class CompactArray; + friend class CompactArray::Reader; + +public: + ALWAYS_INLINE operator UInt8() const + { + if (content_l == content_r) + return read(*content_l); + else + return read(*content_l, *content_r); + } + + Locus ALWAYS_INLINE & operator=(UInt8 content) + { + if ((index_l == index_r) || (index_l == (BITSET_SIZE - 1))) + { + /// Ячейка полностью влезает в один байт. + *content_l &= ~(((1 << content_width) - 1) << offset_l); + *content_l |= content << offset_l; + } + else + { + /// Ячейка перекрывает два байта. + size_t left = 8 - offset_l; + + *content_l &= ~(((1 << left) - 1) << offset_l); + *content_l |= (content & ((1 << left) - 1)) << offset_l; + + *content_r &= ~((1 << offset_r) - 1); + *content_r |= content >> left; + } + + return *this; + } + +private: + Locus() = default; + + Locus(BucketIndex bucket_index) + { + init(bucket_index); + } + + void ALWAYS_INLINE init(BucketIndex bucket_index) + { + size_t l = static_cast(bucket_index) * content_width; + index_l = l >> 3; + offset_l = l & 7; + + size_t r = static_cast(bucket_index + 1) * content_width; + index_r = r >> 3; + offset_r = r & 7; + } + + UInt8 ALWAYS_INLINE read(UInt8 value_l) const + { + /// Ячейка полностью влезает в один байт. + return (value_l >> offset_l) & ((1 << content_width) - 1); + } + + UInt8 ALWAYS_INLINE read(UInt8 value_l, UInt8 value_r) const + { + /// Ячейка перекрывает два байта. + return ((value_l >> offset_l) & ((1 << (8 - offset_l)) - 1)) + | ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l)); + } + +private: + size_t index_l; + size_t offset_l; + size_t index_r; + size_t offset_r; + + UInt8 * content_l; + UInt8 * content_r; + + /// Проверки + static_assert((content_width > 0) && (content_width < 8), "Invalid parameter value"); + static_assert(bucket_count <= (std::numeric_limits::max() / content_width), "Invalid parameter value"); +}; + +} + diff --git a/dbms/include/DB/Common/HyperLogLogCounter.h b/dbms/include/DB/Common/HyperLogLogCounter.h index 685fe2c089e..d6bf0cf9790 100644 --- a/dbms/include/DB/Common/HyperLogLogCounter.h +++ b/dbms/include/DB/Common/HyperLogLogCounter.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -61,154 +62,6 @@ template struct MinCounterType >::Type Type; }; -/** Компактный массив для хранения данных, размер L, в битах, которых составляет меньше одного байта. - * Вместо того, чтобы хранить каждое значение в 8-битную ячейку памяти, что приводит к растрате - * 37.5% пространства для L=5, CompactArray хранит смежные L-битные значения, именно компактные - * ячейки в массиве байтов, т.е. фактически CompactArray симулирует массив L-битных значений. - */ -template -class __attribute__ ((packed)) CompactArray final -{ -public: - class Locus; - -public: - CompactArray() = default; - - UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const - { - Locus locus(bucket_index); - - if (locus.index_l == locus.index_r) - return locus.read(bitset[locus.index_l]); - else - return locus.read(bitset[locus.index_l], bitset[locus.index_r]); - } - - Locus ALWAYS_INLINE operator[](BucketIndex bucket_index) - { - Locus locus(bucket_index); - - locus.content_l = &bitset[locus.index_l]; - - if (locus.index_l == locus.index_r) - locus.content_r = locus.content_l; - else - locus.content_r = &bitset[locus.index_r]; - - return locus; - } - - void readText(DB::ReadBuffer & in) - { - for (size_t i = 0; i < BITSET_SIZE; ++i) - { - if (i != 0) - DB::assertString(",", in); - DB::readIntText(bitset[i], in); - } - } - - void writeText(DB::WriteBuffer & out) const - { - for (size_t i = 0; i < BITSET_SIZE; ++i) - { - if (i != 0) - writeCString(",", out); - DB::writeIntText(bitset[i], out); - } - } - -private: - /// число байт в битсете - static constexpr size_t BITSET_SIZE = (static_cast(bucket_count) * content_width + 7) / 8; - UInt8 bitset[BITSET_SIZE] = { 0 }; -}; - -/** Структура Locus содержит необходимую информацию, чтобы найти для каждой компактной ячейки - * соответствующие физическую ячейку и смещение, в битах, от начала ячейки. Поскольку в общем - * случае размер одной физической ячейки не делится на размер одной компактной ячейки, возможны - * случаи, когда одна компактная ячейка перекрывает две физические ячейки. Поэтому структура - * Locus содержит две пары (индекс, смещение). - */ -template -class CompactArray::Locus final -{ - friend class CompactArray; - -public: - ALWAYS_INLINE operator UInt8() const - { - if (content_l == content_r) - return read(*content_l); - else - return read(*content_l, *content_r); - } - - Locus ALWAYS_INLINE & operator=(UInt8 content) - { - if ((index_l == index_r) || (index_l == (BITSET_SIZE - 1))) - { - /// Компактная ячейка полностью влезает в одну физическую ячейку. - *content_l &= ~(((1 << content_width) - 1) << offset_l); - *content_l |= content << offset_l; - } - else - { - /// Компактная ячейка перекрывает две физические ячейки. - size_t left = 8 - offset_l; - - *content_l &= ~(((1 << left) - 1) << offset_l); - *content_l |= (content & ((1 << left) - 1)) << offset_l; - - *content_r &= ~((1 << offset_r) - 1); - *content_r |= content >> left; - } - - return *this; - } - -private: - Locus() = default; - - Locus(BucketIndex bucket_index) - { - size_t l = static_cast(bucket_index) * content_width; - index_l = l >> 3; - offset_l = l & 7; - - size_t r = static_cast(bucket_index + 1) * content_width; - index_r = r >> 3; - offset_r = r & 7; - } - - UInt8 ALWAYS_INLINE read(UInt8 value_l) const - { - /// Компактная ячейка полностью влезает в одну физическую ячейку. - return (value_l >> offset_l) & ((1 << content_width) - 1); - } - - UInt8 ALWAYS_INLINE read(UInt8 value_l, UInt8 value_r) const - { - /// Компактная ячейка перекрывает две физические ячейки. - return ((value_l >> offset_l) & ((1 << (8 - offset_l)) - 1)) - | ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l)); - } - -private: - size_t index_l; - size_t offset_l; - size_t index_r; - size_t offset_r; - - UInt8 * content_l; - UInt8 * content_r; - - /// Проверки - static_assert((content_width > 0) && (content_width < 8), "Invalid parameter value"); - static_assert(bucket_count <= (std::numeric_limits::max() / content_width), "Invalid parameter value"); -}; - /** Знаменатель формулы алгоритма HyperLogLog */ template; + using RankStore = DB::CompactArray; public: void insert(Value_t value) @@ -476,12 +329,11 @@ public: void readAndMerge(DB::ReadBuffer & in) { - RankStore other; - in.readStrict(reinterpret_cast(&other), sizeof(RankStore)); - for (HashValueType bucket = 0; bucket < bucket_count; ++bucket) + typename RankStore::Reader reader(in); + while (reader.next()) { - UInt8 rank = other[bucket]; - update(bucket, rank); + const auto & data = reader.get(); + update(data.first, data.second); } in.ignore(sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType)); diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index c0aa6030748..a8445610850 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -297,6 +297,8 @@ namespace ErrorCodes MONGODB_INIT_FAILED = 293, INVALID_BLOCK_EXTRA_INFO = 294, RECEIVED_EMPTY_DATA = 295, + NO_REMOTE_SHARD_FOUND = 296, + SHARD_HAS_NO_CONNECTIONS = 297, KEEPER_EXCEPTION = 999, POCO_EXCEPTION = 1000, diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index ae7ade63bff..1dfc844a2f4 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -97,43 +97,195 @@ struct ConvertImpl } }; +/// Реализация функции toDate. -/** Преобразование даты-с-временем в дату: отбрасывание времени. - */ -template -struct ConvertImpl +namespace details { namespace { + +template class Transformation> +class Transformer { - typedef DataTypeDateTime::FieldType FromFieldType; - typedef DataTypeDate::FieldType ToFieldType; +private: + using Op = Transformation; - static void execute(Block & block, const ColumnNumbers & arguments, size_t result) +public: + static void vector_vector(const PODArray & vec_from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, PODArray & vec_to) { - const auto & date_lut = DateLUT::instance(); + ColumnString::Offset_t prev_offset = 0; - if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + for (size_t i = 0; i < vec_from.size(); ++i) { - ColumnVector * col_to = new ColumnVector; - block.getByPosition(result).column = col_to; - - const typename ColumnVector::Container_t & vec_from = col_from->getData(); - typename ColumnVector::Container_t & vec_to = col_to->getData(); - size_t size = vec_from.size(); - vec_to.resize(size); - - for (size_t i = 0; i < size; ++i) - vec_to[i] = date_lut.toDayNum(vec_from[i]); + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + const auto & remote_date_lut = DateLUT::instance(time_zone); + vec_to[i] = Op::execute(vec_from[i], remote_date_lut); + prev_offset = cur_offset; } - else if (const ColumnConst * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + } + + static void vector_constant(const PODArray & vec_from, const std::string & data, + PODArray & vec_to) + { + const auto & remote_date_lut = DateLUT::instance(data); + for (size_t i = 0; i < vec_from.size(); ++i) + vec_to[i] = Op::execute(vec_from[i], remote_date_lut); + } + + static void vector_constant(const PODArray & vec_from, PODArray & vec_to) + { + const auto & local_date_lut = DateLUT::instance(); + for (size_t i = 0; i < vec_from.size(); ++i) + vec_to[i] = Op::execute(vec_from[i], local_date_lut); + } + + static void constant_vector(const FromType & from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, PODArray & vec_to) + { + ColumnString::Offset_t prev_offset = 0; + + for (size_t i = 0; i < offsets.size(); ++i) { - block.getByPosition(result).column = new ColumnConst(col_from->size(), date_lut.toDayNum(col_from->getData())); + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + const auto & remote_date_lut = DateLUT::instance(time_zone); + vec_to[i] = Op::execute(from, remote_date_lut); + prev_offset = cur_offset; } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + Name::name, - ErrorCodes::ILLEGAL_COLUMN); + } + + static void constant_constant(const FromType & from, const std::string & data, ToType & to) + { + const auto & remote_date_lut = DateLUT::instance(data); + to = Op::execute(from, remote_date_lut); + } + + static void constant_constant(const FromType & from, ToType & to) + { + const auto & local_date_lut = DateLUT::instance(); + to = Op::execute(from, local_date_lut); } }; +template class Transformation, typename Name> +class ToDateConverter +{ +private: + using FromFieldType = typename FromType::FieldType; + using ToFieldType = typename DataTypeDate::FieldType; + using Op = Transformer; + +public: + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + const auto * sources = typeid_cast *>(&*source_col); + const auto * const_source = typeid_cast *>(&*source_col); + + if (arguments.size() == 1) + { + if (sources) + { + auto * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const auto & vec_from = sources->getData(); + auto & vec_to = col_to->getData(); + size_t size = vec_from.size(); + vec_to.resize(size); + + Op::vector_constant(vec_from, vec_to); + } + else if (const_source) + { + ToFieldType res; + Op::constant_constant(const_source->getData(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else if (arguments.size() == 2) + { + const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; + const auto * time_zones = typeid_cast(&*time_zone_col); + const auto * const_time_zone = typeid_cast(&*time_zone_col); + + if (sources) + { + auto * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_from = sources->getData(); + auto & vec_to = col_to->getData(); + vec_to.resize(vec_from.size()); + + if (time_zones) + Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); + else if (const_time_zone) + Op::vector_constant(vec_from, const_time_zone->getData(), vec_to); + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else if (const_source) + { + if (time_zones) + { + auto * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_to = col_to->getData(); + vec_to.resize(time_zones->getOffsets().size()); + + Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); + } + else if (const_time_zone) + { + ToFieldType res; + Op::constant_constant(const_source->getData(), const_time_zone->getData(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else + throw Exception("FunctionsConversion: Internal error", ErrorCodes::LOGICAL_ERROR); + } +}; + +template +struct ToDateTransform +{ + static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut) + { + return date_lut.toDayNum(from); + } +}; + +template +struct ToDateTransform32Or64 +{ + static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut) + { + return (from < 0xFFFF) ? from : date_lut.toDayNum(from); + } +}; + +}} + +/** Преобразование даты-с-временем в дату: отбрасывание времени. + */ +template struct ConvertImpl : details::ToDateConverter {}; /** Отдельный случай для преобразования (U)Int32 или (U)Int64 в Date. * Если число меньше 65536, то оно понимается, как DayNum, а если больше или равно - как unix timestamp. @@ -142,56 +294,10 @@ struct ConvertImpl * когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату * (иначе такое использование было бы распространённой ошибкой). */ -template -struct ConvertImpl32Or64ToDate -{ - typedef typename FromDataType::FieldType FromFieldType; - typedef DataTypeDate::FieldType ToFieldType; - - template - static To convert(const From & from, const DateLUTImpl & date_lut) - { - return from < 0xFFFF - ? from - : date_lut.toDayNum(from); - } - - static void execute(Block & block, const ColumnNumbers & arguments, size_t result) - { - const auto & date_lut = DateLUT::instance(); - - if (const ColumnVector * col_from - = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) - { - ColumnVector * col_to = new ColumnVector; - block.getByPosition(result).column = col_to; - - const typename ColumnVector::Container_t & vec_from = col_from->getData(); - typename ColumnVector::Container_t & vec_to = col_to->getData(); - size_t size = vec_from.size(); - vec_to.resize(size); - - for (size_t i = 0; i < size; ++i) - vec_to[i] = convert(vec_from[i], date_lut); - } - else if (const ColumnConst * col_from - = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) - { - block.getByPosition(result).column = new ColumnConst(col_from->size(), - convert(col_from->getData(), date_lut)); - } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + Name::name, - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; - +template struct ConvertImpl : details::ToDateConverter {}; +template struct ConvertImpl : details::ToDateConverter {}; +template struct ConvertImpl : details::ToDateConverter {}; +template struct ConvertImpl : details::ToDateConverter {}; /** Преобразование чисел, дат, дат-с-временем в строки: через форматирование. */ @@ -887,6 +993,8 @@ struct ConvertImpl } }; +/// Предварительное объявление. +struct NameToDate { static constexpr auto name = "toDate"; }; template class FunctionConvert : public IFunction @@ -934,7 +1042,9 @@ public: private: template DataTypePtr getReturnTypeImpl(const DataTypes & arguments, - typename std::enable_if::value || std::is_same::value), void>::type * = nullptr) const + typename std::enable_if::value || + std::is_same::value || + std::is_same::value)>::type * = nullptr) const { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -960,7 +1070,7 @@ private: + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + else if ((arguments.size() == 2) && (typeid_cast(&*arguments[1]) == nullptr)) { throw Exception{ "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), @@ -987,7 +1097,7 @@ private: + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + else if ((arguments.size() == 2) && (typeid_cast(&*arguments[1]) == nullptr)) { throw Exception{ "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), @@ -997,6 +1107,26 @@ private: return new ToDataType2; } + + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + typename std::enable_if::value>::type * = nullptr) const + { + if ((arguments.size() < 1) || (arguments.size() > 2)) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1 or 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if ((arguments.size() == 2) && (typeid_cast(&*arguments[1]) == nullptr)) + { + throw Exception{ + "Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new ToDataType2; + } }; @@ -1140,7 +1270,6 @@ struct NameToInt32 { static constexpr auto name = "toInt32"; }; struct NameToInt64 { static constexpr auto name = "toInt64"; }; struct NameToFloat32 { static constexpr auto name = "toFloat32"; }; struct NameToFloat64 { static constexpr auto name = "toFloat64"; }; -struct NameToDate { static constexpr auto name = "toDate"; }; struct NameToDateTime { static constexpr auto name = "toDateTime"; }; struct NameToString { static constexpr auto name = "toString"; }; diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 9d270044810..f831f37edb4 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -402,49 +402,8 @@ public: return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const override - { - return getReturnTypeImpl(arguments); - } - - /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) override - { - IDataType * from_type = &*block.getByPosition(arguments[0]).type; - - if (typeid_cast(from_type)) - DateTimeTransformImpl::execute(block, arguments, result); - else if (typeid_cast(from_type)) - DateTimeTransformImpl::execute(block, arguments, result); - else - throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - -private: /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - - template - DataTypePtr getReturnTypeImpl(const DataTypes & arguments, - typename std::enable_if< - !(std::is_same::value - || (std::is_same::value && std::is_same::value)) - , void>::type * = nullptr) const - { - if (arguments.size() != 1) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - return new ToDataType; - } - - template - DataTypePtr getReturnTypeImpl(const DataTypes & arguments, - typename std::enable_if< - std::is_same::value - || (std::is_same::value && std::is_same::value) - , void>::type * = nullptr) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if ((arguments.size() < 1) || (arguments.size() > 2)) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -468,6 +427,20 @@ private: return new ToDataType; } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override + { + IDataType * from_type = &*block.getByPosition(arguments[0]).type; + + if (typeid_cast(from_type)) + DateTimeTransformImpl::execute(block, arguments, result); + else if (typeid_cast(from_type)) + DateTimeTransformImpl::execute(block, arguments, result); + else + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } }; diff --git a/dbms/include/DB/IO/ReadHelpers.h b/dbms/include/DB/IO/ReadHelpers.h index 71fb8f4c9f3..e8a2412d5ec 100644 --- a/dbms/include/DB/IO/ReadHelpers.h +++ b/dbms/include/DB/IO/ReadHelpers.h @@ -427,7 +427,7 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf); inline void readDateTimeText(time_t & datetime, ReadBuffer & buf) { /** Считываем 10 символов, которые могут быть unix timestamp. - * При этом, поддерживается только unix timestamp из 10 символов - от 9 сентября 2001. + * При этом, поддерживается только unix timestamp из 5-10 символов. * Потом смотрим на пятый символ. Если это число - парсим unix timestamp. * Если это не число - парсим YYYY-MM-DD hh:mm:ss. */ diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index c3e6a2f05f7..785af306698 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -968,6 +968,25 @@ protected: Table & table_dst, Table & table_src) const; + /// Слить данные из хэш-таблицы src в dst, но только для ключей, которые уже есть в dst. В остальных случаях, слить данные в overflows. + template + void mergeDataNoMoreKeysImpl( + Table & table_dst, + AggregatedDataWithoutKey & overflows, + Table & table_src) const; + + /// То же самое, но игнорирует остальные ключи. + template + void mergeDataOnlyExistingKeysImpl( + Table & table_dst, + Table & table_src) const; + + /// Слить все ключи, оставшиеся после предыдущего метода, в overflows. + template + void mergeDataRemainingKeysToOverflowsImpl( + AggregatedDataWithoutKey & overflows, + Table & table_src) const; + void mergeWithoutKeyDataImpl( ManyAggregatedDataVariants & non_empty_data) const; @@ -1024,13 +1043,24 @@ protected: bool final, boost::threadpool::pool * thread_pool) const; + template + void mergeStreamsImplCase( + Block & block, + const Sizes & key_sizes, + Arena * aggregates_pool, + Method & method, + Table & data, + AggregateDataPtr overflow_row) const; + template void mergeStreamsImpl( Block & block, const Sizes & key_sizes, Arena * aggregates_pool, Method & method, - Table & data) const; + Table & data, + AggregateDataPtr overflow_row, + bool no_more_keys) const; void mergeWithoutKeyStreamsImpl( Block & block, @@ -1049,6 +1079,15 @@ protected: template void destroyImpl( Method & method) const; + + + /** Проверяет ограничения на максимальное количество ключей для агрегации. + * Если оно превышено, то, в зависимости от group_by_overflow_mode, либо + * - кидает исключение; + * - возвращает false, что говорит о том, что выполнение нужно прервать; + * - выставляет переменную no_more_keys в true. + */ + bool checkLimits(size_t result_size, bool & no_more_keys) const; }; diff --git a/dbms/include/DB/Interpreters/Cluster.h b/dbms/include/DB/Interpreters/Cluster.h index 074b039d0ea..50f16137f0c 100644 --- a/dbms/include/DB/Interpreters/Cluster.h +++ b/dbms/include/DB/Interpreters/Cluster.h @@ -13,7 +13,7 @@ namespace DB /// С локальными узлами соединение не устанавливается, а выполяется запрос напрямую. /// Поэтому храним только количество локальных узлов /// В конфиге кластер включает в себя узлы или -class Cluster : private boost::noncopyable +class Cluster { public: Cluster(const Settings & settings, const String & cluster_name); @@ -22,28 +22,13 @@ public: Cluster(const Settings & settings, std::vector> names, const String & username, const String & password); - /// количество узлов clickhouse сервера, расположенных локально - /// к локальным узлам обращаемся напрямую - size_t getLocalNodesNum() const { return local_nodes_num; } + Cluster(const Cluster &) = delete; + Cluster & operator=(const Cluster &) = delete; /// используеться для выставления ограничения на размер таймаута static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit); public: - /// Соединения с удалёнными серверами. - ConnectionPools pools; - - struct ShardInfo - { - /// contains names of directories for asynchronous write to StorageDistributed - std::vector dir_names; - UInt32 shard_num; - int weight; - size_t num_local_nodes; - }; - std::vector shard_info_vec; - std::vector slot_to_shard; - struct Address { /** В конфиге адреса либо находятся в узлах : @@ -73,26 +58,59 @@ public: Address(const String & host_port_, const String & user_, const String & password_); }; -private: - static bool isLocal(const Address & address); + using Addresses = std::vector
; + using AddressesWithFailover = std::vector; + + struct ShardInfo + { + public: + bool isLocal() const { return !local_addresses.empty(); } + bool hasRemoteConnections() const { return !pool.isNull(); } + size_t getLocalNodeCount() const { return local_addresses.size(); } + + public: + /// contains names of directories for asynchronous write to StorageDistributed + std::vector dir_names; + UInt32 shard_num; + int weight; + Addresses local_addresses; + mutable ConnectionPoolPtr pool; + }; + + using ShardsInfo = std::vector; public: - /// Массив шардов. Каждый шард - адреса одного сервера. - typedef std::vector
Addresses; + const ShardsInfo & getShardsInfo() const { return shards_info; } + const Addresses & getShardsAddresses() const { return addresses; } + const AddressesWithFailover & getShardsWithFailoverAddresses() const { return addresses_with_failover; } - /// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными). - typedef std::vector AddressesWithFailover; + const ShardInfo * getAnyRemoteShardInfo() const { return any_remote_shard_info; } - const Addresses & getShardsInfo() const { return addresses; } - const AddressesWithFailover & getShardsWithFailoverInfo() const { return addresses_with_failover; } - const Addresses & getLocalShardsInfo() const { return local_addresses; } + /// Количество удалённых шардов. + size_t getRemoteShardCount() const { return remote_shard_count; } + + /// Количество узлов clickhouse сервера, расположенных локально + /// к локальным узлам обращаемся напрямую. + size_t getLocalShardCount() const { return local_shard_count; } + +public: + std::vector slot_to_shard; private: - Addresses addresses; - AddressesWithFailover addresses_with_failover; - Addresses local_addresses; + void initMisc(); - size_t local_nodes_num = 0; +private: + /// Описание шардов кластера. + ShardsInfo shards_info; + /// Любой удалённый шард. + ShardInfo * any_remote_shard_info = nullptr; + /// Массив шардов. Каждый шард - адреса одного сервера. + Addresses addresses; + /// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными). + AddressesWithFailover addresses_with_failover; + + size_t remote_shard_count = 0; + size_t local_shard_count = 0; }; struct Clusters diff --git a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h index d7111b1345f..d7a1350b4f5 100644 --- a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h +++ b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h @@ -182,6 +182,9 @@ private: /// Например, для ARRAY JOIN [1,2] AS b сюда попадет "b" -> "array(1,2)". NameToNameMap array_join_alias_to_name; + /// Обратное отображение для array_join_alias_to_name. + NameToNameMap array_join_name_to_alias; + /// Нужно ли подготавливать к выполнению глобальные подзапросы при анализировании запроса. bool do_global; diff --git a/dbms/include/DB/Parsers/ParserCheckQuery.h b/dbms/include/DB/Parsers/ParserCheckQuery.h index bc747890657..cba07d953b8 100644 --- a/dbms/include/DB/Parsers/ParserCheckQuery.h +++ b/dbms/include/DB/Parsers/ParserCheckQuery.h @@ -1,13 +1,13 @@ #pragma once -#include +#include namespace DB { /** Запрос вида * CHECK [TABLE] [database.]table */ -class ParserCheckQuery : public IParserBase +class ParserCheckQuery : public ParserQueryWithOutput { protected: const char * getName() const { return "ALTER query"; } diff --git a/dbms/include/DB/Parsers/ParserQueryWithOutput.h b/dbms/include/DB/Parsers/ParserQueryWithOutput.h new file mode 100644 index 00000000000..f1a2bd24b2b --- /dev/null +++ b/dbms/include/DB/Parsers/ParserQueryWithOutput.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/** Парсер для запросов поддерживающих секцию FORMAT. + */ +class ParserQueryWithOutput : public IParserBase +{ +protected: + bool parseFormat(ASTQueryWithOutput & query, Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected); + +protected: + ParserWhiteSpaceOrComments ws; +}; + +} diff --git a/dbms/include/DB/Parsers/ParserSelectQuery.h b/dbms/include/DB/Parsers/ParserSelectQuery.h index 06f918b83ad..bda7d72b35d 100644 --- a/dbms/include/DB/Parsers/ParserSelectQuery.h +++ b/dbms/include/DB/Parsers/ParserSelectQuery.h @@ -1,13 +1,13 @@ #pragma once -#include +#include namespace DB { -class ParserSelectQuery : public IParserBase +class ParserSelectQuery : public ParserQueryWithOutput { protected: const char * getName() const { return "SELECT query"; } diff --git a/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h b/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h index 1c77ae0eac3..d40f9bc4079 100644 --- a/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h +++ b/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -12,7 +13,7 @@ namespace DB /** Запрос SHOW PROCESSLIST */ -class ParserShowProcesslistQuery : public IParserBase +class ParserShowProcesslistQuery : public ParserQueryWithOutput { protected: const char * getName() const { return "SHOW PROCESSLIST query"; } @@ -21,12 +22,11 @@ protected: { Pos begin = pos; - ParserWhiteSpaceOrComments ws; ParserString s_show("SHOW", true, true); ParserString s_processlist("PROCESSLIST", true, true); - ParserString s_format("FORMAT", true, true); - ASTPtr format; + ASTShowProcesslistQuery * query = new ASTShowProcesslistQuery; + ASTPtr query_ptr = query; ws.ignore(pos, end); @@ -40,22 +40,12 @@ protected: ws.ignore(pos, end); - if (s_format.ignore(pos, end, max_parsed_pos, expected)) - { - ws.ignore(pos, end); + /// FORMAT format_name + if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected)) + return false; - ParserIdentifier format_p; - - if (!format_p.parse(pos, end, format, max_parsed_pos, expected)) - return false; - typeid_cast(*format).kind = ASTIdentifier::Format; - - ws.ignore(pos, end); - } - - ASTShowProcesslistQuery * query = new ASTShowProcesslistQuery(StringRange(begin, pos)); - query->format = format; - node = query; + query->range = StringRange(begin, pos); + node = query_ptr; return true; } diff --git a/dbms/include/DB/Parsers/ParserShowTablesQuery.h b/dbms/include/DB/Parsers/ParserShowTablesQuery.h index 264ca5824d6..e88cfa0009c 100644 --- a/dbms/include/DB/Parsers/ParserShowTablesQuery.h +++ b/dbms/include/DB/Parsers/ParserShowTablesQuery.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -11,7 +11,7 @@ namespace DB * или * SHOW DATABASES. */ -class ParserShowTablesQuery : public IParserBase +class ParserShowTablesQuery : public ParserQueryWithOutput { protected: const char * getName() const { return "SHOW TABLES|DATABASES query"; } diff --git a/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h b/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h index fe4d6868a6a..066a46c2283 100644 --- a/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h +++ b/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -9,7 +10,7 @@ namespace DB /** Запрос (EXISTS | SHOW CREATE | (DESCRIBE | DESC) ) [TABLE] [db.]name [FORMAT format] */ -class ParserTablePropertiesQuery : public IParserBase +class ParserTablePropertiesQuery : public ParserQueryWithOutput { protected: const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; } diff --git a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h index 318316242ce..38601adf9b4 100644 --- a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h @@ -39,7 +39,7 @@ public: void write(const Block & block) override { - if (storage.getShardingKeyExpr() && storage.cluster.shard_info_vec.size() > 1) + if (storage.getShardingKeyExpr() && (storage.cluster.getShardsInfo().size() > 1)) return writeSplit(block); writeImpl(block); @@ -50,7 +50,7 @@ private: static std::vector createFiltersImpl(const size_t num_rows, const IColumn * column, const Cluster & cluster) { const auto total_weight = cluster.slot_to_shard.size(); - const auto num_shards = cluster.shard_info_vec.size(); + const auto num_shards = cluster.getShardsInfo().size(); std::vector filters(num_shards); /** Деление отрицательного числа с остатком на положительное, в C++ даёт отрицательный остаток. @@ -123,7 +123,7 @@ private: auto filters = createFilters(block); - const auto num_shards = storage.cluster.shard_info_vec.size(); + const auto num_shards = storage.cluster.getShardsInfo().size(); for (size_t i = 0; i < num_shards; ++i) { auto target_block = block.cloneEmpty(); @@ -138,9 +138,9 @@ private: void writeImpl(const Block & block, const size_t shard_id = 0) { - const auto & shard_info = storage.cluster.shard_info_vec[shard_id]; - if (shard_info.num_local_nodes) - writeToLocal(block, shard_info.num_local_nodes); + const auto & shard_info = storage.cluster.getShardsInfo()[shard_id]; + if (shard_info.getLocalNodeCount() > 0) + writeToLocal(block, shard_info.getLocalNodeCount()); /// dir_names is empty if shard has only local addresses if (!shard_info.dir_names.empty()) diff --git a/dbms/include/DB/TableFunctions/TableFunctionRemote.h b/dbms/include/DB/TableFunctions/TableFunctionRemote.h index 894fc4d5cc0..e454ad6df6f 100644 --- a/dbms/include/DB/TableFunctions/TableFunctionRemote.h +++ b/dbms/include/DB/TableFunctions/TableFunctionRemote.h @@ -132,10 +132,15 @@ private: Settings settings = context.getSettings(); NamesAndTypesList res; - /// Отправляем на первый попавшийся шард + /// Отправляем на первый попавшийся удалённый шард. + const auto shard_info = cluster.getAnyRemoteShardInfo(); + if (shard_info == nullptr) + throw Exception("No remote shard found", ErrorCodes::NO_REMOTE_SHARD_FOUND); + ConnectionPoolPtr pool = shard_info->pool; + BlockInputStreamPtr input{ new RemoteBlockInputStream{ - cluster.pools.front().get(), query, &settings, nullptr, + pool.get(), query, &settings, nullptr, Tables(), QueryProcessingStage::Complete, context} }; input->readPrefix(); diff --git a/dbms/src/Common/tests/compact_array.cpp b/dbms/src/Common/tests/compact_array.cpp new file mode 100644 index 00000000000..ed5d4863bc5 --- /dev/null +++ b/dbms/src/Common/tests/compact_array.cpp @@ -0,0 +1,261 @@ +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +namespace fs = boost::filesystem; + +std::string createTmpPath(const std::string & filename) +{ + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = mkdtemp(pattern); + if (dir == nullptr) + throw std::runtime_error("Could not create directory"); + + return std::string(dir) + "/" + filename; +} + +template +struct Test +{ + static void perform() + { + bool ok = true; + + std::string filename; + + try + { + using Store = DB::CompactArray; + + Store store; + + for (size_t i = 0; i < bucket_count; ++i) + store[i] = Generator::execute(i, width); + + filename = createTmpPath("compact_array.bin"); + + { + DB::WriteBufferFromFile wb(filename); + wb.write(reinterpret_cast(&store), sizeof(store)); + const unsigned char * p = reinterpret_cast(&store); + for (size_t i = 0; i < sizeof(store); ++i) + ++p; + } + + { + DB::ReadBufferFromFile rb(filename); + typename Store::Reader reader(rb); + while (reader.next()) + { + const auto & data = reader.get(); + if (data.second != store[data.first]) + throw std::runtime_error("Found discrepancy"); + } + } + } + catch (const Poco::Exception & ex) + { + std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " failed " + << "(Error: " << ex.what() << ": " << ex.displayText() << ")\n"; + ok = false; + } + catch (const std::runtime_error & ex) + { + std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " failed " + << "(Error: " << ex.what() << ")\n"; + ok = false; + } + catch (...) + { + std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " failed\n"; + ok = false; + } + + fs::remove_all(fs::path(filename).parent_path().string()); + + if (ok) + std::cout << "Test width=" << width << " bucket_count=" << bucket_count << " passed\n"; + } +}; + +template +struct TestSet +{ + static void execute() + { + Test<1, 1, Generator>::perform(); + Test<1, 2, Generator>::perform(); + Test<1, 3, Generator>::perform(); + Test<1, 4, Generator>::perform(); + Test<1, 5, Generator>::perform(); + Test<1, 6, Generator>::perform(); + Test<1, 7, Generator>::perform(); + Test<1, 8, Generator>::perform(); + Test<1, 9, Generator>::perform(); + Test<1, 10, Generator>::perform(); + Test<1, 16, Generator>::perform(); + Test<1, 32, Generator>::perform(); + Test<1, 64, Generator>::perform(); + Test<1, 128, Generator>::perform(); + Test<1, 256, Generator>::perform(); + Test<1, 512, Generator>::perform(); + Test<1, 1024, Generator>::perform(); + + Test<2, 1, Generator>::perform(); + Test<2, 2, Generator>::perform(); + Test<2, 3, Generator>::perform(); + Test<2, 4, Generator>::perform(); + Test<2, 5, Generator>::perform(); + Test<2, 6, Generator>::perform(); + Test<2, 7, Generator>::perform(); + Test<2, 8, Generator>::perform(); + Test<2, 9, Generator>::perform(); + Test<2, 10, Generator>::perform(); + Test<2, 16, Generator>::perform(); + Test<2, 32, Generator>::perform(); + Test<2, 64, Generator>::perform(); + Test<2, 128, Generator>::perform(); + Test<2, 256, Generator>::perform(); + Test<2, 512, Generator>::perform(); + Test<2, 1024, Generator>::perform(); + + Test<3, 1, Generator>::perform(); + Test<3, 2, Generator>::perform(); + Test<3, 3, Generator>::perform(); + Test<3, 4, Generator>::perform(); + Test<3, 5, Generator>::perform(); + Test<3, 6, Generator>::perform(); + Test<3, 7, Generator>::perform(); + Test<3, 8, Generator>::perform(); + Test<3, 9, Generator>::perform(); + Test<3, 10, Generator>::perform(); + Test<3, 16, Generator>::perform(); + Test<3, 32, Generator>::perform(); + Test<3, 64, Generator>::perform(); + Test<3, 128, Generator>::perform(); + Test<3, 256, Generator>::perform(); + Test<3, 512, Generator>::perform(); + Test<3, 1024, Generator>::perform(); + + Test<4, 1, Generator>::perform(); + Test<4, 2, Generator>::perform(); + Test<4, 3, Generator>::perform(); + Test<4, 4, Generator>::perform(); + Test<4, 5, Generator>::perform(); + Test<4, 6, Generator>::perform(); + Test<4, 7, Generator>::perform(); + Test<4, 8, Generator>::perform(); + Test<4, 9, Generator>::perform(); + Test<4, 10, Generator>::perform(); + Test<4, 16, Generator>::perform(); + Test<4, 32, Generator>::perform(); + Test<4, 64, Generator>::perform(); + Test<4, 128, Generator>::perform(); + Test<4, 256, Generator>::perform(); + Test<4, 512, Generator>::perform(); + Test<4, 1024, Generator>::perform(); + + Test<5, 1, Generator>::perform(); + Test<5, 2, Generator>::perform(); + Test<5, 3, Generator>::perform(); + Test<5, 4, Generator>::perform(); + Test<5, 5, Generator>::perform(); + Test<5, 6, Generator>::perform(); + Test<5, 7, Generator>::perform(); + Test<5, 8, Generator>::perform(); + Test<5, 9, Generator>::perform(); + Test<5, 10, Generator>::perform(); + Test<5, 16, Generator>::perform(); + Test<5, 32, Generator>::perform(); + Test<5, 64, Generator>::perform(); + Test<5, 128, Generator>::perform(); + Test<5, 256, Generator>::perform(); + Test<5, 512, Generator>::perform(); + Test<5, 1024, Generator>::perform(); + + Test<6, 1, Generator>::perform(); + Test<6, 2, Generator>::perform(); + Test<6, 3, Generator>::perform(); + Test<6, 4, Generator>::perform(); + Test<6, 5, Generator>::perform(); + Test<6, 6, Generator>::perform(); + Test<6, 7, Generator>::perform(); + Test<6, 8, Generator>::perform(); + Test<6, 9, Generator>::perform(); + Test<6, 10, Generator>::perform(); + Test<6, 16, Generator>::perform(); + Test<6, 32, Generator>::perform(); + Test<6, 64, Generator>::perform(); + Test<6, 128, Generator>::perform(); + Test<6, 256, Generator>::perform(); + Test<6, 512, Generator>::perform(); + Test<6, 1024, Generator>::perform(); + + Test<7, 1, Generator>::perform(); + Test<7, 2, Generator>::perform(); + Test<7, 3, Generator>::perform(); + Test<7, 4, Generator>::perform(); + Test<7, 5, Generator>::perform(); + Test<7, 6, Generator>::perform(); + Test<7, 7, Generator>::perform(); + Test<7, 8, Generator>::perform(); + Test<7, 9, Generator>::perform(); + Test<7, 10, Generator>::perform(); + Test<7, 16, Generator>::perform(); + Test<7, 32, Generator>::perform(); + Test<7, 64, Generator>::perform(); + Test<7, 128, Generator>::perform(); + Test<7, 256, Generator>::perform(); + Test<7, 512, Generator>::perform(); + Test<7, 1024, Generator>::perform(); + } +}; + +struct Generator1 +{ + static UInt8 execute(size_t i, size_t width) + { + return (1 << width) - 1; + } +}; + +struct Generator2 +{ + static UInt8 execute(size_t i, size_t width) + { + return (i >> 1) & ((1 << width) - 1); + } +}; + +struct Generator3 +{ + static UInt8 execute(size_t i, size_t width) + { + return (i * 17 + 31) % (1ULL << width); + } +}; + +void runTests() +{ + std::cout << "Test set 1\n"; + TestSet::execute(); + std::cout << "Test set 2\n"; + TestSet::execute(); + std::cout << "Test set 3\n"; + TestSet::execute(); +} + +int main() +{ + runTests(); + return 0; +} diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index dcbf140b55f..83730a9b5e2 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -265,21 +265,28 @@ void readBackQuotedString(String & s, ReadBuffer & buf) void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf) { - char s[19]; + static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19; + static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10; - size_t size = buf.read(s, 10); - if (10 != size) + char s[DATE_TIME_BROKEN_DOWN_LENGTH]; + char * s_pos = s; + + /// Кусок, похожий на unix timestamp. + while (s_pos < s + UNIX_TIMESTAMP_MAX_LENGTH && !buf.eof() && *buf.position() >= '0' && *buf.position() <= '9') { - s[size] = 0; - throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); + *s_pos = *buf.position(); + ++s_pos; + ++buf.position(); } - if (s[4] < '0' || s[4] > '9') + /// 2015-01-01 01:02:03 + if (s_pos == s + 4 && !buf.eof() && (*buf.position() < '0' || *buf.position() > '9')) { - size_t size = buf.read(&s[10], 9); - if (9 != size) + const size_t remaining_size = DATE_TIME_BROKEN_DOWN_LENGTH - (s_pos - s); + size_t size = buf.read(s_pos, remaining_size); + if (remaining_size != size) { - s[10 + size] = 0; + s_pos[size] = 0; throw Exception(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); } @@ -297,7 +304,7 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf) datetime = DateLUT::instance().makeDateTime(year, month, day, hour, minute, second); } else - datetime = parse(s, 10); + datetime = parse(s, s_pos - s); } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 4b8e2dc4885..7e9445a524e 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -715,6 +715,15 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result, result.convertToTwoLevel(); /// Проверка ограничений. + if (!checkLimits(result_size, no_more_keys)) + return false; + + return true; +} + + +bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const +{ if (!no_more_keys && max_rows_to_group_by && result_size > max_rows_to_group_by) { if (group_by_overflow_mode == OverflowMode::THROW) @@ -1226,6 +1235,86 @@ void NO_INLINE Aggregator::mergeDataImpl( } +template +void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl( + Table & table_dst, + AggregatedDataWithoutKey & overflows, + Table & table_src) const +{ + for (auto it = table_src.begin(); it != table_src.end(); ++it) + { + decltype(it) res_it = table_dst.find(it->first, it.getHash()); + + AggregateDataPtr res_data = table_dst.end() == res_it + ? overflows + : Method::getAggregateData(res_it->second); + + for (size_t i = 0; i < aggregates_size; ++i) + aggregate_functions[i]->merge( + res_data + offsets_of_aggregate_states[i], + Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + + for (size_t i = 0; i < aggregates_size; ++i) + aggregate_functions[i]->destroy( + Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + + Method::getAggregateData(it->second) = nullptr; + } +} + +template +void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl( + Table & table_dst, + Table & table_src) const +{ + for (auto it = table_src.begin(); it != table_src.end(); ++it) + { + decltype(it) res_it = table_dst.find(it->first, it.getHash()); + + if (table_dst.end() == res_it) + continue; + + AggregateDataPtr res_data = Method::getAggregateData(res_it->second); + + for (size_t i = 0; i < aggregates_size; ++i) + aggregate_functions[i]->merge( + res_data + offsets_of_aggregate_states[i], + Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + + for (size_t i = 0; i < aggregates_size; ++i) + aggregate_functions[i]->destroy( + Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + + Method::getAggregateData(it->second) = nullptr; + } +} + +template +void NO_INLINE Aggregator::mergeDataRemainingKeysToOverflowsImpl( + AggregatedDataWithoutKey & overflows, + Table & table_src) const +{ + for (auto it = table_src.begin(); it != table_src.end(); ++it) + { + if (Method::getAggregateData(it->second) == nullptr) + continue; + + AggregateDataPtr res_data = overflows; + + for (size_t i = 0; i < aggregates_size; ++i) + aggregate_functions[i]->merge( + res_data + offsets_of_aggregate_states[i], + Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + + for (size_t i = 0; i < aggregates_size; ++i) + aggregate_functions[i]->destroy( + Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + + Method::getAggregateData(it->second) = nullptr; + } +} + + void NO_INLINE Aggregator::mergeWithoutKeyDataImpl( ManyAggregatedDataVariants & non_empty_data) const { @@ -1253,15 +1342,25 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl( ManyAggregatedDataVariants & non_empty_data) const { AggregatedDataVariantsPtr & res = non_empty_data[0]; + bool no_more_keys = false; /// Все результаты агрегации соединяем с первым. for (size_t i = 1, size = non_empty_data.size(); i < size; ++i) { + if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys)) + break; + AggregatedDataVariants & current = *non_empty_data[i]; - mergeDataImpl( - getDataVariant(*res).data, - getDataVariant(current).data); + if (!no_more_keys) + mergeDataImpl( + getDataVariant(*res).data, + getDataVariant(current).data); + else + mergeDataNoMoreKeysImpl( + getDataVariant(*res).data, + res->without_key, + getDataVariant(current).data); /// current не будет уничтожать состояния агрегатных функций в деструкторе current.aggregator = nullptr; @@ -1276,8 +1375,13 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl( { AggregatedDataVariantsPtr & res = non_empty_data[0]; + /// В данном случае, no_more_keys будет выставлено, только если в первом (самом большом) состоянии достаточно много строк. + bool no_more_keys = false; + if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys)) + return; + /// Слияние распараллеливается по корзинам - первому уровню TwoLevelHashMap. - auto merge_bucket = [&non_empty_data, &res, this](size_t bucket, MemoryTracker * memory_tracker) + auto merge_bucket = [&non_empty_data, &res, no_more_keys, this](size_t bucket, MemoryTracker * memory_tracker) { current_memory_tracker = memory_tracker; @@ -1286,12 +1390,18 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl( { AggregatedDataVariants & current = *non_empty_data[i]; - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], - getDataVariant(current).data.impls[bucket]); - - /// current не будет уничтожать состояния агрегатных функций в деструкторе - current.aggregator = nullptr; + if (!no_more_keys) + { + mergeDataImpl( + getDataVariant(*res).data.impls[bucket], + getDataVariant(current).data.impls[bucket]); + } + else + { + mergeDataOnlyExistingKeysImpl( + getDataVariant(*res).data.impls[bucket], + getDataVariant(current).data.impls[bucket]); + } } }; @@ -1326,6 +1436,25 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl( for (auto & task : tasks) if (task.valid()) task.get_future().get(); + + if (no_more_keys && overflow_row) + { + for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) + { + for (size_t i = 1, size = non_empty_data.size(); i < size; ++i) + { + AggregatedDataVariants & current = *non_empty_data[i]; + + mergeDataRemainingKeysToOverflowsImpl( + res->without_key, + getDataVariant(current).data.impls[bucket]); + } + } + } + + /// aggregator не будет уничтожать состояния агрегатных функций в деструкторе + for (size_t i = 1, size = non_empty_data.size(); i < size; ++i) + non_empty_data[i]->aggregator = nullptr; } @@ -1350,6 +1479,13 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va if (non_empty_data.size() == 1) return non_empty_data[0]; + /// Отсортируем состояния по убыванию размера, чтобы мердж был более эффективным (так как все состояния мерджатся в первое). + std::sort(non_empty_data.begin(), non_empty_data.end(), + [](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs) + { + return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow(); + }); + /// Если хотя бы один из вариантов двухуровневый, то переконвертируем все варианты в двухуровневые, если есть не такие. /// Замечание - возможно, было бы более оптимально не конвертировать одноуровневые варианты перед мерджем, а мерджить их отдельно, в конце. @@ -1448,13 +1584,14 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va } -template -void NO_INLINE Aggregator::mergeStreamsImpl( +template +void NO_INLINE Aggregator::mergeStreamsImplCase( Block & block, const Sizes & key_sizes, Arena * aggregates_pool, Method & method, - Table & data) const + Table & data, + AggregateDataPtr overflow_row) const { ConstColumnPlainPtrs key_columns(keys_size); AggregateColumnsData aggregate_columns(aggregates_size); @@ -1475,13 +1612,33 @@ void NO_INLINE Aggregator::mergeStreamsImpl( for (size_t i = 0; i < rows; ++i) { typename Table::iterator it; + bool inserted; /// Вставили новый ключ, или такой ключ уже был? + bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys. /// Получаем ключ для вставки в хэш-таблицу. auto key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *aggregates_pool); - data.emplace(key, it, inserted); + if (!no_more_keys) + { + data.emplace(key, it, inserted); + } + else + { + inserted = false; + it = data.find(key); + if (data.end() == it) + overflow = true; + } + /// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего. + if (no_more_keys && overflow && !overflow_row) + { + method.onExistingKey(key, keys, *aggregates_pool); + continue; + } + + /// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом. if (inserted) { AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); @@ -1496,10 +1653,12 @@ void NO_INLINE Aggregator::mergeStreamsImpl( else method.onExistingKey(key, keys, *aggregates_pool); + AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row; + /// Мерджим состояния агрегатных функций. for (size_t j = 0; j < aggregates_size; ++j) aggregate_functions[j]->merge( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[j], + value + offsets_of_aggregate_states[j], (*aggregate_columns[j])[i]); } @@ -1507,6 +1666,23 @@ void NO_INLINE Aggregator::mergeStreamsImpl( block.clear(); } +template +void NO_INLINE Aggregator::mergeStreamsImpl( + Block & block, + const Sizes & key_sizes, + Arena * aggregates_pool, + Method & method, + Table & data, + AggregateDataPtr overflow_row, + bool no_more_keys) const +{ + if (!no_more_keys) + mergeStreamsImplCase(block, key_sizes, aggregates_pool, method, data, overflow_row); + else + mergeStreamsImplCase(block, key_sizes, aggregates_pool, method, data, overflow_row); +} + + void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( Block & block, AggregatedDataVariants & result) const @@ -1621,6 +1797,11 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants /// Сначала параллельно мерджим для отдельных bucket-ов. Затем домердживаем данные, не распределённые по bucket-ам. if (has_two_level) { + /** В этом случае, no_more_keys не поддерживается в связи с тем, что + * из разных потоков трудно обновлять общее состояние для "остальных" ключей (overflows). + * То есть, ключей в итоге может оказаться существенно больше, чем max_rows_to_group_by. + */ + LOG_TRACE(log, "Merging partially aggregated two-level data."); auto merge_bucket = [&bucket_to_blocks, &result, &key_sizes, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker) @@ -1634,7 +1815,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants #define M(NAME) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]); + mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false); if (false) {} APPLY_FOR_VARIANTS_TWO_LEVEL(M) @@ -1691,6 +1872,8 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants { LOG_TRACE(log, "Merging partially aggregated single-level data."); + bool no_more_keys = false; + BlocksList & blocks = bucket_to_blocks[-1]; for (Block & block : blocks) { @@ -1700,12 +1883,15 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants return; } + if (!checkLimits(result.sizeWithoutOverflowRow(), no_more_keys)) + break; + if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows) mergeWithoutKeyStreamsImpl(block, result); #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data); + mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys); APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M @@ -1760,7 +1946,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data); + mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false); APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 025780f1ae7..a4704059a28 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -8,34 +8,67 @@ namespace DB { +namespace +{ + +/// Вес шарда по-умолчанию. +static constexpr int default_weight = 1; + +inline bool isLocal(const Cluster::Address & address) +{ + /// Если среди реплик существует такая, что: + /// - её порт совпадает с портом, который слушает сервер; + /// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера + /// то нужно всегда ходить на этот шард без межпроцессного взаимодействия + return isLocalAddress(address.resolved_address); +} + +inline std::string addressToDirName(const Cluster::Address & address) +{ + return + escapeForFileName(address.user) + + (address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' + + escapeForFileName(address.resolved_address.host().toString()) + ':' + + std::to_string(address.resolved_address.port()); +} + +inline bool beginsWith(const std::string & str1, const char * str2) +{ + if (str2 == nullptr) + throw Exception("Passed null pointer to function beginsWith", ErrorCodes::LOGICAL_ERROR); + return 0 == strncmp(str1.data(), str2, strlen(str2)); +} /// Для кэширования DNS запросов. -static Poco::Net::SocketAddress resolveSocketAddressImpl1(const String & host, UInt16 port) +Poco::Net::SocketAddress resolveSocketAddressImpl1(const String & host, UInt16 port) { return Poco::Net::SocketAddress(host, port); } -static Poco::Net::SocketAddress resolveSocketAddressImpl2(const String & host_and_port) +Poco::Net::SocketAddress resolveSocketAddressImpl2(const String & host_and_port) { return Poco::Net::SocketAddress(host_and_port); } -static Poco::Net::SocketAddress resolveSocketAddress(const String & host, UInt16 port) +Poco::Net::SocketAddress resolveSocketAddress(const String & host, UInt16 port) { static SimpleCache cache; return cache(host, port); } -static Poco::Net::SocketAddress resolveSocketAddress(const String & host_and_port) +Poco::Net::SocketAddress resolveSocketAddress(const String & host_and_port) { static SimpleCache cache; return cache(host_and_port); } +} + +/// Реализация класса Cluster::Address Cluster::Address::Address(const String & config_prefix) { - auto & config = Poco::Util::Application::instance().config(); + const auto & config = Poco::Util::Application::instance().config(); host_name = config.getString(config_prefix + ".host"); port = config.getInt(config_prefix + ".port"); @@ -51,7 +84,7 @@ Cluster::Address::Address(const String & host_port_, const String & user_, const UInt16 default_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0); /// Похоже на то, что строка host_port_ содержит порт. Если условие срабатывает - не обязательно значит, что порт есть (пример: [::]). - if (nullptr != strchr(host_port_.c_str(), ':') || !default_port) + if ((nullptr != strchr(host_port_.c_str(), ':')) || !default_port) { resolved_address = resolveSocketAddress(host_port_); host_name = host_port_.substr(0, host_port_.find(':')); @@ -65,19 +98,7 @@ Cluster::Address::Address(const String & host_port_, const String & user_, const } } - -namespace -{ - inline std::string addressToDirName(const Cluster::Address & address) - { - return - escapeForFileName(address.user) + - (address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' + - escapeForFileName(address.resolved_address.host().toString()) + ':' + - std::to_string(address.resolved_address.port()); - } -} - +/// Реализация класса Clusters Clusters::Clusters(const Settings & settings, const String & config_name) { @@ -85,17 +106,16 @@ Clusters::Clusters(const Settings & settings, const String & config_name) Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_name, config_keys); - for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) + for (const auto & key : config_keys) impl.emplace(std::piecewise_construct, - std::forward_as_tuple(*it), - std::forward_as_tuple(settings, config_name + "." + *it)); + std::forward_as_tuple(key), + std::forward_as_tuple(settings, config_name + "." + key)); } +/// Реализация класса Cluster Cluster::Cluster(const Settings & settings, const String & cluster_name) { - /// Создать кластер. - Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config(); Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(cluster_name, config_keys); @@ -104,35 +124,56 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name) UInt32 current_shard_num = 1; - for (auto it = config_keys.begin(); it != config_keys.end(); ++it) + for (const auto & key : config_keys) { - if (0 == strncmp(it->c_str(), "node", strlen("node"))) + if (beginsWith(key, "node")) { - const auto & prefix = config_prefix + *it; - const auto weight = config.getInt(prefix + ".weight", 1); + /// Шард без реплик. + + const auto & prefix = config_prefix + key; + const auto weight = config.getInt(prefix + ".weight", default_weight); if (weight == 0) continue; addresses.emplace_back(prefix); addresses.back().replica_num = 1; + const auto & address = addresses.back(); - slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size()); - if (const auto is_local = isLocal(addresses.back())) - shard_info_vec.push_back({{}, current_shard_num, weight, is_local}); + ShardInfo info; + info.shard_num = current_shard_num; + info.weight = weight; + + if (isLocal(address)) + info.local_addresses.push_back(address); else - shard_info_vec.push_back({{addressToDirName(addresses.back())}, current_shard_num, weight, is_local}); + { + info.dir_names.push_back(addressToDirName(address)); + info.pool = new ConnectionPool( + settings.distributed_connections_pool_size, + address.host_name, address.port, address.resolved_address, + "", address.user, address.password, + "server", Protocol::Compression::Enable, + saturate(settings.connect_timeout, settings.limits.max_execution_time), + saturate(settings.receive_timeout, settings.limits.max_execution_time), + saturate(settings.send_timeout, settings.limits.max_execution_time)); + } + + slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size()); + shards_info.push_back(info); } - else if (0 == strncmp(it->c_str(), "shard", strlen("shard"))) + else if (beginsWith(key, "shard")) { + /// Шард с репликами. + Poco::Util::AbstractConfiguration::Keys replica_keys; - config.keys(config_prefix + *it, replica_keys); + config.keys(config_prefix + key, replica_keys); addresses_with_failover.emplace_back(); Addresses & replica_addresses = addresses_with_failover.back(); UInt32 current_replica_num = 1; - const auto & partial_prefix = config_prefix + *it + "."; - const auto weight = config.getInt(partial_prefix + ".weight", 1); + const auto & partial_prefix = config_prefix + key + "."; + const auto weight = config.getInt(partial_prefix + ".weight", default_weight); if (weight == 0) continue; @@ -142,26 +183,20 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name) * the first element of vector; otherwise we will just .emplace_back */ std::vector dir_names{}; - size_t num_local_nodes = 0; auto first = true; - for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt) + for (const auto & replica_key : replica_keys) { - if (0 == strncmp(jt->data(), "weight", strlen("weight")) || - 0 == strncmp(jt->data(), "internal_replication", strlen("internal_replication"))) + if (beginsWith(replica_key, "weight") || beginsWith(replica_key, "internal_replication")) continue; - if (0 == strncmp(jt->c_str(), "replica", strlen("replica"))) + if (beginsWith(replica_key, "replica")) { - replica_addresses.emplace_back(partial_prefix + *jt); + replica_addresses.emplace_back(partial_prefix + replica_key); replica_addresses.back().replica_num = current_replica_num; ++current_replica_num; - if (isLocal(replica_addresses.back())) - { - ++num_local_nodes; - } - else + if (!isLocal(replica_addresses.back())) { if (internal_replication) { @@ -178,40 +213,18 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name) } } else - throw Exception("Unknown element in config: " + *jt, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + throw Exception("Unknown element in config: " + replica_key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } - slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size()); - shard_info_vec.push_back({std::move(dir_names), current_shard_num, weight, num_local_nodes}); - } - else - throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + Addresses shard_local_addresses; - ++current_shard_num; - } - - /// Создать соответствующие пулы соединений. - - if (!addresses_with_failover.empty() && !addresses.empty()) - throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); - - if (!addresses_with_failover.empty()) - { - for (const auto & shard : addresses_with_failover) - { ConnectionPools replicas; - replicas.reserve(shard.size()); + replicas.reserve(replica_addresses.size()); - bool has_local_replica = false; - - for (const auto & replica : shard) + for (const auto & replica : replica_addresses) { if (isLocal(replica)) - { - has_local_replica = true; - local_addresses.push_back(replica); - break; - } + shard_local_addresses.push_back(replica); else { replicas.emplace_back(new ConnectionPool( @@ -225,42 +238,31 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name) } } - if (has_local_replica) - ++local_nodes_num; - else - pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries)); + ConnectionPoolPtr shard_pool; + if (!replicas.empty()) + shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries); + + slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size()); + shards_info.push_back({std::move(dir_names), current_shard_num, weight, shard_local_addresses, shard_pool}); } + else + throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + + if (!addresses_with_failover.empty() && !addresses.empty()) + throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + + ++current_shard_num; } - else if (!addresses.empty()) - { - for (const auto & address : addresses) - { - if (isLocal(address)) - { - local_addresses.push_back(address); - ++local_nodes_num; - } - else - { - pools.emplace_back(new ConnectionPool( - settings.distributed_connections_pool_size, - address.host_name, address.port, address.resolved_address, - "", address.user, address.password, - "server", Protocol::Compression::Enable, - saturate(settings.connect_timeout, settings.limits.max_execution_time), - saturate(settings.receive_timeout, settings.limits.max_execution_time), - saturate(settings.send_timeout, settings.limits.max_execution_time))); - } - } - } - else - throw Exception("No addresses listed in config", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + initMisc(); } Cluster::Cluster(const Settings & settings, std::vector> names, const String & username, const String & password) { + UInt32 current_shard_num = 1; + for (const auto & shard : names) { Addresses current; @@ -284,8 +286,14 @@ Cluster::Cluster(const Settings & settings, std::vector> nam saturate(settings.send_timeout, settings.limits.max_execution_time))); } - pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries)); + ConnectionPoolPtr shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries); + + slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size()); + shards_info.push_back({{}, current_shard_num, default_weight, {}, shard_pool}); + ++current_shard_num; } + + initMisc(); } @@ -294,17 +302,35 @@ Poco::Timespan Cluster::saturate(const Poco::Timespan & v, const Poco::Timespan if (limit.totalMicroseconds() == 0) return v; else - return v > limit ? limit : v; + return (v > limit) ? limit : v; } -bool Cluster::isLocal(const Address & address) +void Cluster::initMisc() { - /// Если среди реплик существует такая, что: - /// - её порт совпадает с портом, который слушает сервер; - /// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера - /// то нужно всегда ходить на этот шард без межпроцессного взаимодействия - return isLocalAddress(address.resolved_address); + for (const auto & shard_info : shards_info) + { + if (!shard_info.isLocal() && !shard_info.hasRemoteConnections()) + throw Exception("Found shard without any specified connection", + ErrorCodes::SHARD_HAS_NO_CONNECTIONS); + } + + for (const auto & shard_info : shards_info) + { + if (shard_info.isLocal()) + ++local_shard_count; + else + ++remote_shard_count; + } + + for (auto & shard_info : shards_info) + { + if (!shard_info.isLocal()) + { + any_remote_shard_info = &shard_info; + break; + } + } } } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 86f2647a01f..e93bea468ab 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -148,9 +148,9 @@ void ExpressionAction::prepare(Block & sample_block) case ARRAY_JOIN: { - for (NameSet::iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it) + for (const auto & name : array_joined_columns) { - ColumnWithTypeAndName & current = sample_block.getByName(*it); + ColumnWithTypeAndName & current = sample_block.getByName(name); const DataTypeArray * array_type = typeid_cast(&*current.type); if (!array_type) throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); @@ -214,6 +214,7 @@ void ExpressionAction::prepare(Block & sample_block) } } + void ExpressionAction::execute(Block & block) const { // std::cerr << "executing: " << toString() << std::endl; @@ -261,9 +262,11 @@ void ExpressionAction::execute(Block & block) const { if (array_joined_columns.empty()) throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR); + ColumnPtr any_array_ptr = block.getByName(*array_joined_columns.begin()).column; if (any_array_ptr->isConst()) any_array_ptr = dynamic_cast(*any_array_ptr).convertToFullColumn(); + const ColumnArray * any_array = typeid_cast(&*any_array_ptr); if (!any_array) throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH); @@ -552,19 +555,22 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre { if (actions.empty() || actions.back().type != ExpressionAction::ARRAY_JOIN) return false; + NameSet required_set(required_columns.begin(), required_columns.end()); + for (const std::string & name : actions.back().array_joined_columns) - { if (required_set.count(name)) return false; - } + for (const std::string & name : actions.back().array_joined_columns) { DataTypePtr & type = sample_block.getByName(name).type; type = new DataTypeArray(type); } + out_action = actions.back(); actions.pop_back(); + return true; } @@ -1015,7 +1021,11 @@ void ExpressionActionsChain::finalize() steps[i].actions->finalize(required_output); } - /// Когда возможно, перенесем ARRAY JOIN из более ранних шагов в более поздние. + /** Когда возможно, перенесем ARRAY JOIN из более ранних шагов в более поздние. + * Замечание: обычно это полезно, так как ARRAY JOIN - сложная операция, которая, как правило, увеличивает объём данных. + * Но не всегда - в случае, если большинство массивов пустые, ARRAY JOIN, наоборот, уменьшает объём данных, + * и его было бы полезно делать раньше. Этот случай не рассматривается. + */ for (size_t i = 1; i < steps.size(); ++i) { ExpressionAction action; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index b825225e4a0..71b4fe794b5 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -211,13 +211,17 @@ void ExpressionAnalyzer::analyzeAggregation() /// constant expressions have non-null column pointer at this stage if (const auto is_constexpr = col.column) { - if (i < group_asts.size() - 1) - group_asts[i] = std::move(group_asts.back()); + /// but don't remove last key column if no aggregate functions, otherwise aggregation will not work + if (!aggregate_descriptions.empty() || group_asts.size() > 1) + { + if (i < group_asts.size() - 1) + group_asts[i] = std::move(group_asts.back()); - group_asts.pop_back(); - i -= 1; + group_asts.pop_back(); + i -= 1; - continue; + continue; + } } NameAndTypePair key{column_name, col.type}; @@ -781,7 +785,26 @@ void ExpressionAnalyzer::optimizeGroupBy() } if (group_exprs.empty()) - select_query->group_expression_list = nullptr; + { + /** Нельзя полностью убирать GROUP BY. Потому что если при этом даже агрегатных функций не было, то получится, что не будет агрегации. + * Вместо этого оставим GROUP BY const. + * Далее см. удаление констант в методе analyzeAggregation. + */ + + /// Нужно вставить константу, которая не является именем столбца таблицы. Такой случай редкий, но бывает. + UInt64 unused_column = 0; + String unused_column_name = toString(unused_column); + + while (columns.end() != std::find_if(columns.begin(), columns.end(), + [&unused_column_name](const NameAndTypePair & name_type) { return name_type.name == unused_column_name; })) + { + ++unused_column; + unused_column_name = toString(unused_column); + } + + select_query->group_expression_list = new ASTExpressionList; + select_query->group_expression_list->children.push_back(new ASTLiteral(StringRange(), UInt64(unused_column))); + } } @@ -1384,12 +1407,15 @@ void ExpressionAnalyzer::getArrayJoinedColumns() { const String nested_table_name = ast->getColumnName(); const String nested_table_alias = ast->getAliasOrColumnName(); + if (nested_table_alias == nested_table_name && !typeid_cast(&*ast)) throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED); if (array_join_alias_to_name.count(nested_table_alias) || aliases.count(nested_table_alias)) throw Exception("Duplicate alias " + nested_table_alias, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); + array_join_alias_to_name[nested_table_alias] = nested_table_name; + array_join_name_to_alias[nested_table_name] = nested_table_alias; } ASTs & query_asts = select_query->children; @@ -1432,6 +1458,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns() } +/// Заполняет array_join_result_to_source: по каким столбцам-массивам размножить, и как их после этого назвать. void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) { if (ASTIdentifier * node = typeid_cast(&*ast)) @@ -1439,14 +1466,29 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) if (node->kind == ASTIdentifier::Column) { String table_name = DataTypeNested::extractNestedTableName(node->name); + if (array_join_alias_to_name.count(node->name)) - array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; + { + /// Был написан ARRAY JOIN со столбцом-массивом. Пример: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1 + array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1 + } else if (array_join_alias_to_name.count(table_name)) { - String nested_column = DataTypeNested::extractNestedColumnName(node->name); - array_join_result_to_source[node->name] + /// Был написан ARRAY JOIN с вложенной таблицей. Пример: SELECT PP.Key1 FROM ... ARRAY JOIN ParsedParams AS PP + String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1 + array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1 = DataTypeNested::concatenateNestedName(array_join_alias_to_name[table_name], nested_column); } + else if (array_join_name_to_alias.count(table_name)) + { + /** Пример: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP. + * То есть, в запросе используется исходный массив, размноженный по самому себе. + */ + + String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1 + array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1 + DataTypeNested::concatenateNestedName(array_join_name_to_alias[table_name], nested_column)] = node->name; + } } } else @@ -1488,10 +1530,12 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl if (node->kind == ASTFunction::LAMBDA_EXPRESSION) throw Exception("Unexpected expression", ErrorCodes::UNEXPECTED_EXPRESSION); + /// Функция arrayJoin. if (node->kind == ASTFunction::ARRAY_JOIN) { if (node->arguments->children.size() != 1) throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH); + ASTPtr arg = node->arguments->children.at(0); getActionsImpl(arg, no_subqueries, only_consts, actions_stack); if (!only_consts) @@ -1787,13 +1831,17 @@ void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAn } } +/// "Большой" ARRAY JOIN. void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const { NameSet result_columns; for (const auto & result_source : array_join_result_to_source) { + /// Дать столбцам новые имена, если надо. if (result_source.first != result_source.second) actions->add(ExpressionAction::copyColumn(result_source.second, result_source.first)); + + /// Сделать ARRAY JOIN (заменить массивы на их внутренности) для столбцов в этими новыми именами. result_columns.insert(result_source.first); } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 01aea41fc95..260312bcde4 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -33,6 +33,7 @@ #include #include #include +#include namespace DB @@ -331,18 +332,21 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns( /// specific code for different data types, e.g. toFixedString(col, N) for DataTypeFixedString if (const auto fixed_string = typeid_cast(data_type_ptr)) { - const auto conversion_function_name = "toFixedString"; - default_expr_list->children.emplace_back(setAlias( makeASTFunction( - conversion_function_name, + "toFixedString", ASTPtr{new ASTIdentifier{{}, tmp_column_name}}, ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}), final_column_name)); } + else if (typeid_cast(data_type_ptr)) + { + /// do not perform conversion on arrays, require exact type match + default_expr_list->children.emplace_back(setAlias( + col_decl.default_expression->clone(), final_column_name)); + } else { - /// @todo fix for parametric types, results in broken code, i.e. toArray(ElementType)(col) const auto conversion_function_name = "to" + data_type_ptr->getName(); default_expr_list->children.emplace_back(setAlias( @@ -370,15 +374,29 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns( 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"); + const auto & column_name = col_decl_ptr->name; + const auto has_explicit_type = nullptr != col_decl_ptr->type; + auto & explicit_type = name_and_type_ptr->type; - /// type mismatch between explicitly specified and deduced type, add conversion - if (name_and_type_ptr->type->getName() != tmp_column.type->getName()) + /// if column declaration contains explicit type, name_and_type_ptr->type is not null + if (has_explicit_type) + { + const auto & tmp_column = block.getByName(column_name + "_tmp"); + const auto & deduced_type = tmp_column.type; + + /// type mismatch between explicitly specified and deduced type, add conversion for non-array types + if (explicit_type->getName() != deduced_type->getName()) { - col_decl_ptr->default_expression = makeASTFunction( - "to" + name_and_type_ptr->type->getName(), + /// foolproof against defaulting array columns incorrectly + if (typeid_cast(explicit_type.get())) + throw Exception{ + "Default expression type mismatch for column " + column_name + + ". Expected " + explicit_type->getName() + ", deduced " + + deduced_type->getName(), + ErrorCodes::TYPE_MISMATCH + }; + + col_decl_ptr->default_expression = makeASTFunction("to" + explicit_type->getName(), col_decl_ptr->default_expression); col_decl_ptr->children.clear(); @@ -387,9 +405,10 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns( } } else - name_and_type_ptr->type = block.getByName(name_and_type_ptr->name).type; + /// no explicit type, name_and_type_ptr->type is null, set to deduced type + explicit_type = block.getByName(column_name).type; - defaults.emplace(col_decl_ptr->name, ColumnDefault{ + defaults.emplace(column_name, ColumnDefault{ columnDefaultTypeFromString(col_decl_ptr->default_specifier), col_decl_ptr->default_expression }); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7c3d43b63c3..19663c550e8 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -471,6 +471,17 @@ void InterpreterSelectQuery::executeSingleQuery() /// Если есть агрегация, выполняем выражения в SELECT и ORDER BY на инициировавшем сервере, иначе - на серверах-источниках. query_analyzer->appendSelect(chain, need_aggregate ? !second_stage : !first_stage); selected_columns = chain.getLastStep().required_output; + + /** Если есть LIMIT, то ARRAY JOIN нельзя переносить позже ORDER BY + * (так как он меняет количество строк и частичная сортировка ORDER с LIMIT-ом перестают правильно работать). + */ + if (query.order_expression_list && query.limit_length && query.array_join_expression_list) + { + /// Завершаем цепочку действий, в рамках которой ARRAY JOIN может переноситься. + chain.finalize(); + chain.clear(); + } + has_order_by = query_analyzer->appendOrderBy(chain, need_aggregate ? !second_stage : !first_stage); before_order_and_select = chain.getLastActions(); chain.addStep(); diff --git a/dbms/src/ODBC/README b/dbms/src/ODBC/README new file mode 100644 index 00000000000..f81f09029d2 --- /dev/null +++ b/dbms/src/ODBC/README @@ -0,0 +1,16 @@ +Install unixodbc. + +g++ -std=gnu++1y -Wall -g -shared -fPIC -lPocoFoundation -lPocoNet -o odbc.so odbc.cpp + +~/.odbc.ini: + +[ClickHouse] +Driver = /home/milovidov/work/metrika-core/metrica/src/dbms/src/ODBC/odbc.so +Description = ClickHouse driver +DATABASE = default +HOST = localhost +PORT = 9000 +FRAMED = 0 + +Run +iusql -v ClickHouse diff --git a/dbms/src/ODBC/odbc.cpp b/dbms/src/ODBC/odbc.cpp new file mode 100644 index 00000000000..e83c2361687 --- /dev/null +++ b/dbms/src/ODBC/odbc.cpp @@ -0,0 +1,1750 @@ +#include +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +static void mylog(const char * message) +{ + static struct Once + { + Once() + { + std::string stderr_path = "/tmp/clickhouse-odbc-stderr"; + if (!freopen(stderr_path.c_str(), "a+", stderr)) + throw std::logic_error("Cannot freopen stderr."); + } + } once; + + std::cerr << message << "\n"; +} + + +struct StringRef +{ + const char * data = nullptr; + size_t size = 0; + + StringRef() {} + StringRef(const char * c_str) { *this = c_str; } + StringRef & operator= (const char * c_str) { data = c_str; size = strlen(c_str); return *this; } + + std::string toString() const { return {data, size}; } + + bool operator== (const char * rhs) const + { + return size == strlen(rhs) && 0 == memcmp(data, rhs, strlen(rhs)); + } + + operator bool() const { return data != nullptr; } +}; + + +/// Парсит строку вида key1=value1;key2=value2... TODO Парсинг значений в фигурных скобках. +static const char * nextKeyValuePair(const char * data, const char * end, StringRef & out_key, StringRef & out_value) +{ + if (data >= end) + return nullptr; + + const char * key_begin = data; + const char * key_end = reinterpret_cast(memchr(key_begin, '=', end - key_begin)); + if (!key_end) + return nullptr; + + const char * value_begin = key_end + 1; + const char * value_end; + if (value_begin >= end) + value_end = value_begin; + else + { + value_end = reinterpret_cast(memchr(value_begin, ';', end - value_begin)); + if (!value_end) + value_end = end; + } + + out_key.data = key_begin; + out_key.size = key_end - key_begin; + + out_value.data = value_begin; + out_value.size = value_end - value_begin; + + if (value_end < end && *value_end == ';') + return value_end + 1; + return value_end; +} + + +template +RETCODE fillOutputString(const char * value, size_t size_without_zero, + PTR out_value, LENGTH out_value_max_length, LENGTH * out_value_length) +{ + if (out_value_length) + *out_value_length = size_without_zero; + + if (out_value_max_length < 0) + return SQL_ERROR; + + bool res = SQL_SUCCESS; + + if (out_value) + { + if (out_value_max_length >= static_cast(size_without_zero + 1)) + { + memcpy(out_value, value, size_without_zero + 1); + } + else + { + if (out_value_max_length > 0) + { + memcpy(out_value, value, out_value_max_length - 1); + reinterpret_cast(out_value)[out_value_max_length - 1] = 0; + + std::cerr << (char*)(out_value) << "\n"; + } + res = SQL_SUCCESS_WITH_INFO; + } + } + + return res; +} + + +struct DiagnosticRecord +{ + SQLINTEGER native_error_code = 0; + std::string sql_state = "-----"; + std::string message; +}; + + +struct Environment +{ + struct TypeInfo + { + std::string sql_type_name; + size_t display_size; + bool is_unsigned; + }; + + const std::map types_info = + { + {"UInt8", { .sql_type_name = "TINYINT", .display_size = 3, .is_unsigned = true, }}, + {"UInt16", { .sql_type_name = "SMALLINT", .display_size = 5, .is_unsigned = true, }}, + {"UInt32", { .sql_type_name = "INT", .display_size = 11, .is_unsigned = true, }}, + {"UInt64", { .sql_type_name = "BIGINT", .display_size = 20, .is_unsigned = true, }}, + {"Int8", { .sql_type_name = "TINYINT", .display_size = 3, .is_unsigned = false, }}, + {"Int16", { .sql_type_name = "SMALLINT", .display_size = 5, .is_unsigned = false, }}, + {"Int32", { .sql_type_name = "INT", .display_size = 11, .is_unsigned = false, }}, + {"Int64", { .sql_type_name = "BIGINT", .display_size = 20, .is_unsigned = false, }}, + {"Float32", { .sql_type_name = "FLOAT", .display_size = 1024, .is_unsigned = false, }}, + {"Float64", { .sql_type_name = "DOUBLE", .display_size = 1024, .is_unsigned = false, }}, + {"String", { .sql_type_name = "TEXT", .display_size = 16777216, .is_unsigned = true, }}, + {"FixedString", { .sql_type_name = "TEXT", .display_size = 256, .is_unsigned = true, }}, + {"Date", { .sql_type_name = "DATE", .display_size = 20, .is_unsigned = true, }}, + {"DateTime", { .sql_type_name = "DATETIME", .display_size = 20, .is_unsigned = true, }}, + {"Array", { .sql_type_name = "TEXT", .display_size = 16777216, .is_unsigned = true, }}, + }; + +/* Poco::UTF8Encoding utf8; + Poco::UTF16Encoding utf16; + Poco::TextConverter converter_utf8_to_utf16 {utf8, utf16};*/ + + int odbc_version = SQL_OV_ODBC3; + DiagnosticRecord diagnostic_record; +}; + + +struct Connection +{ + Connection(Environment & env_) : environment(env_) {} + + Environment & environment; + std::string host = "localhost"; + uint16_t port = 8123; + std::string user = "default"; + std::string password; + std::string database = "default"; + + Poco::Net::HTTPClientSession session; + + DiagnosticRecord diagnostic_record; +}; + + +struct Statement +{ + Statement(Connection & conn_) : connection(conn_) {} + + Connection & connection; + std::string query; + Poco::Net::HTTPRequest request; + Poco::Net::HTTPResponse response; + std::istream * in; + + DiagnosticRecord diagnostic_record; + + struct ColumnInfo + { + std::string name; + std::string type; + }; + + std::vector columns_info; + + void initializeResultSet() + { + /// TODO Обработка исключений, отправленных сервером. + /// TODO Случай отсутствия данных. + while (true) + { + std::string name; + *in >> name; /// TODO Поддержка эскейпленных строк. + + std::cerr << "name: " << name << "\n"; + + if (!in->good()) + throw std::runtime_error("Incomplete header received."); + + ColumnInfo column; + column.name = name; + columns_info.push_back(std::move(column)); + + auto c = in->get(); + if (c == '\n') + break; /// TODO Более корректный код. + } + + size_t i = 0; + size_t size = columns_info.size(); + for (; i < size; ++i) + { + std::string type; + *in >> type; + + std::cerr << "type: " << type << "\n"; + + if (!in->good()) + throw std::runtime_error("Incomplete header received."); + + columns_info[i].type = type; + + auto c = in->get(); + if (c == '\n') + break; + } + + std::cerr << i << ", " << size << "\n"; + + if (i + 1 != size) + throw std::runtime_error("Number of types doesn't equal to number of columns."); + } + + + std::vector current_row; + size_t row_count = 0; + + bool fetchRow() + { + size_t size = columns_info.size(); + if (!size) + return false; + + if (current_row.empty()) + current_row.resize(size); + + size_t i = 0; + for (; i < size; ++i) + { + std::string value; + *in >> value; /// TODO Здесь всё неправильно. + + std::cerr << "value: " << value << "\n"; + + if (!in->good()) + { + if (i == 0) + return false; + else + throw std::runtime_error("Incomplete row received."); + } + + current_row[i] = std::move(value); + + auto c = in->get(); + if (c == '\n') + break; + } + + if (i + 1 != size) + throw std::runtime_error("Number of values in row doesn't equal to number of columns."); + + ++row_count; + return true; + } + + static uint64_t getUInt(const std::string s) + { + return Poco::NumberParser::parseUnsigned64(s); + } + + static int64_t getInt(const std::string s) + { + return Poco::NumberParser::parse64(s); + } + + static float getFloat(const std::string s) + { + return Poco::NumberParser::parseFloat(s); + } + + static double getDouble(const std::string s) + { + return Poco::NumberParser::parseFloat(s); + } +}; + + +RETCODE allocEnv(SQLHENV * out_environment) +{ + if (nullptr == out_environment) + return SQL_INVALID_HANDLE; + + *out_environment = new Environment; + + return SQL_SUCCESS; +} + +RETCODE allocConnect(SQLHENV environment, SQLHDBC * out_connection) +{ + if (nullptr == out_connection) + return SQL_INVALID_HANDLE; + + *out_connection = new Connection(*reinterpret_cast(environment)); + + return SQL_SUCCESS; +} + +RETCODE allocStmt(SQLHDBC connection, SQLHSTMT * out_statement) +{ + if (nullptr == out_statement || nullptr == connection) + return SQL_INVALID_HANDLE; + + *out_statement = new Statement(*reinterpret_cast(connection)); + + return SQL_SUCCESS; +} + +RETCODE freeEnv(SQLHENV environment) +{ + delete reinterpret_cast(environment); + return SQL_SUCCESS; +} + +RETCODE freeConnect(SQLHDBC connection) +{ + delete reinterpret_cast(connection); + return SQL_SUCCESS; +} + +RETCODE freeStmt(SQLHSTMT statement) +{ + delete reinterpret_cast(statement); + return SQL_SUCCESS; +} + + +/// Интерфейс библиотеки. +extern "C" +{ + + +RETCODE SQL_API +SQLAllocHandle(SQLSMALLINT handle_type, + SQLHANDLE input_handle, + SQLHANDLE * output_handle) +{ + mylog(__FUNCTION__); + + switch (handle_type) + { + case SQL_HANDLE_ENV: + return allocEnv((SQLHENV *)output_handle); + case SQL_HANDLE_DBC: + return allocConnect((SQLHENV)input_handle, (SQLHDBC *)output_handle); + case SQL_HANDLE_STMT: + return allocStmt((SQLHDBC)input_handle, (SQLHSTMT *)output_handle); + default: + return SQL_ERROR; + } +} + +RETCODE SQL_API +SQLAllocEnv(SQLHDBC * output_handle) +{ + mylog(__FUNCTION__); + return allocEnv(output_handle); +} + +RETCODE SQL_API +SQLAllocConnect(SQLHENV input_handle, SQLHDBC * output_handle) +{ + mylog(__FUNCTION__); + return allocConnect(input_handle, output_handle); +} + +RETCODE SQL_API +SQLAllocStmt(SQLHDBC input_handle, SQLHSTMT * output_handle) +{ + mylog(__FUNCTION__); + return allocStmt(input_handle, output_handle); +} + + +RETCODE SQL_API +SQLFreeHandle(SQLSMALLINT handleType, SQLHANDLE handle) +{ + mylog(__FUNCTION__); + + switch (handleType) + { + case SQL_HANDLE_ENV: + return freeEnv((SQLHENV)handle); + case SQL_HANDLE_DBC: + return freeConnect((SQLHDBC)handle); + case SQL_HANDLE_STMT: + return freeStmt((SQLHDBC)handle); + default: + return SQL_ERROR; + } +} + + +RETCODE SQL_API +SQLFreeEnv(HENV handle) +{ + mylog(__FUNCTION__); + return freeEnv(handle); +} + +RETCODE SQL_API +SQLFreeConnect(HDBC handle) +{ + mylog(__FUNCTION__); + return freeConnect(handle); +} + +RETCODE SQL_API +SQLFreeStmt(HSTMT statement_handle, + SQLUSMALLINT option) +{ + mylog(__FUNCTION__); + + switch (option) + { + case SQL_DROP: + return freeStmt(statement_handle); + + case SQL_CLOSE: /// Закрыть курсор, проигнорировать оставшиеся результаты. Если курсора нет, то noop. + case SQL_UNBIND: + case SQL_RESET_PARAMS: + return SQL_SUCCESS; + + default: + return SQL_ERROR; + } +} + + +RETCODE SQL_API +SQLConnect(HDBC connection_handle, + SQLCHAR * server_name, SQLSMALLINT server_name_size, + SQLCHAR * user, SQLSMALLINT user_size, + SQLCHAR * password, SQLSMALLINT password_size) +{ + mylog(__FUNCTION__); + + if (nullptr == connection_handle) + return SQL_INVALID_HANDLE; + + Connection & connection = *reinterpret_cast(connection_handle); + + if (connection.session.connected()) + return SQL_ERROR; + + if (user) + { + if (user_size < 0) + user_size = strlen(reinterpret_cast(user)); + connection.user.assign(reinterpret_cast(user), static_cast(user_size)); + } + + if (password) + { + if (password_size < 0) + password_size = strlen(reinterpret_cast(password)); + connection.password.assign(reinterpret_cast(password), static_cast(password_size)); + } + + connection.session.setHost(connection.host); + connection.session.setPort(connection.port); + connection.session.setKeepAlive(true); + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLDriverConnect(HDBC connection_handle, + HWND unused_window, + SQLCHAR FAR * connection_str_in, + SQLSMALLINT connection_str_in_size, + SQLCHAR FAR * connection_str_out, + SQLSMALLINT connection_str_out_max_size, + SQLSMALLINT FAR * connection_str_out_size, + SQLUSMALLINT driver_completion) +{ + mylog(__FUNCTION__); + + if (nullptr == connection_handle) + return SQL_INVALID_HANDLE; + + Connection & connection = *reinterpret_cast(connection_handle); + + if (connection.session.connected()) + return SQL_ERROR; + + if (nullptr == connection_str_in) + return SQL_ERROR; + + /// Почему-то при использовании isql, сюда передаётся -3. TODO С чего бы это? + if (connection_str_in_size < 0) + connection_str_in_size = strlen(reinterpret_cast(connection_str_in)); + + /// connection_str_in - строка вида DSN=ClickHouse;UID=default;PWD=password + + const char * data = reinterpret_cast(connection_str_in); + const char * end = reinterpret_cast(connection_str_in) + connection_str_in_size; + + StringRef current_key; + StringRef current_value; + + while ((data = nextKeyValuePair(data, end, current_key, current_value))) + { + if (current_key == "UID") + connection.user = current_value.toString(); + else if (current_key == "PWD") + connection.password = current_value.toString(); + else if (current_key == "HOST") + connection.host = current_value.toString(); + else if (current_key == "PORT") + { + int int_port = 0; + if (Poco::NumberParser::tryParse(current_value.toString(), int_port)) + connection.port = int_port; + else + return SQL_ERROR; + } + else if (current_key == "DATABASE") + connection.database = current_value.toString(); + } + + connection.session.setHost(connection.host); + connection.session.setPort(connection.port); + connection.session.setKeepAlive(true); + + /// TODO Таймаут. + /// TODO Ловля исключений. + + std::cerr << connection_str_in << "\n"; + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLGetInfo(HDBC connection_handle, + SQLUSMALLINT info_type, + PTR out_info_value, SQLSMALLINT out_info_value_max_length, SQLSMALLINT * out_info_value_length) +{ + mylog(__FUNCTION__); + + std::cerr << "GetInfo with info_type: " << info_type << ", out_info_value_max_length: " << out_info_value_max_length << ", out_info_value: " << (void*)out_info_value << "\n"; + + std::string res; + + switch (info_type) + { + case SQL_DRIVER_VER: + res = "1.0"; + break; + case SQL_DRIVER_ODBC_VER: + res = "03.80"; + break; + case SQL_DRIVER_NAME: + res = "ClickHouse ODBC"; + break; + case SQL_DBMS_NAME: + res = "ClickHouse"; + break; + case SQL_SERVER_NAME: + res = "ClickHouse"; + break; + case SQL_DATA_SOURCE_NAME: + res = "ClickHouse"; + break; + + case SQL_MAX_COLUMNS_IN_SELECT: + case SQL_MAX_DRIVER_CONNECTIONS: + case SQL_MAX_CONCURRENT_ACTIVITIES: + case SQL_MAX_COLUMN_NAME_LEN: + case SQL_MAX_CURSOR_NAME_LEN: + case SQL_MAX_SCHEMA_NAME_LEN: + case SQL_MAX_CATALOG_NAME_LEN: + case SQL_MAX_TABLE_NAME_LEN: + case SQL_MAX_COLUMNS_IN_GROUP_BY: + case SQL_MAX_COLUMNS_IN_INDEX: + case SQL_MAX_COLUMNS_IN_ORDER_BY: + case SQL_MAX_COLUMNS_IN_TABLE: + case SQL_MAX_INDEX_SIZE: + case SQL_MAX_ROW_SIZE: + case SQL_MAX_STATEMENT_LEN: + case SQL_MAX_TABLES_IN_SELECT: + case SQL_MAX_USER_NAME_LEN: + res.assign("\0\0\0\0"); + break; + + case SQL_DATA_SOURCE_READ_ONLY: /// TODO Libreoffice + default: + std::cerr << "Unsupported info type: " << info_type << "\n"; /// TODO Унификация трассировки. + return SQL_ERROR; + } + + return fillOutputString(res.data(), res.size(), out_info_value, out_info_value_max_length, out_info_value_length); +} + + +RETCODE SQL_API +SQLPrepare(HSTMT statement_handle, + SQLCHAR * statement_text, SQLINTEGER statement_text_size) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + if (nullptr == statement_text) + return SQL_ERROR; + + Statement & statement = *reinterpret_cast(statement_handle); + + if (!statement.query.empty()) + return SQL_ERROR; + + if (statement_text_size < 0) /// TODO И снова сюда передаётся -3. С чего бы это? + statement_text_size = strlen(reinterpret_cast(statement_text)); + + statement.query.assign(reinterpret_cast(statement_text), static_cast(statement_text_size)); + + std::cerr << statement.query << "\n"; + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLExecute(HSTMT statement_handle) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + Statement & statement = *reinterpret_cast(statement_handle); + + if (statement.query.empty()) + return SQL_ERROR; + + /// Отправляем запрос на сервер. + + std::ostringstream user_password_base64; + Poco::Base64Encoder base64_encoder(user_password_base64); + base64_encoder << statement.connection.user << ":" << statement.connection.password; /// TODO Проверка, что user не содержит символа :. + base64_encoder.close(); + + statement.request.setMethod(Poco::Net::HTTPRequest::HTTP_POST); + statement.request.setCredentials("Basic", user_password_base64.str()); + statement.request.setURI("/?default_format=TabSeparatedWithNamesAndTypes"); /// TODO Возможность передать настройки. + + statement.connection.session.sendRequest(statement.request) << statement.query; + statement.in = &statement.connection.session.receiveResponse(statement.response); + + statement.initializeResultSet(); + + for (const auto & info : statement.columns_info) + std::cerr << info.name << ", " << info.type << "\n"; + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLExecDirect(HSTMT statement_handle, + SQLCHAR * statement_text, SQLINTEGER statement_text_size) +{ + mylog(__FUNCTION__); + + RETCODE ret = SQLPrepare(statement_handle, statement_text, statement_text_size); + if (ret != SQL_SUCCESS) + return ret; + + return SQLExecute(statement_handle); +} + + +RETCODE SQL_API +SQLNumResultCols(HSTMT statement_handle, + SQLSMALLINT * column_count) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + if (nullptr == column_count) + return SQL_ERROR; + + *column_count = reinterpret_cast(statement_handle)->columns_info.size(); + std::cerr << *column_count << "\n"; + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLColAttribute(HSTMT statement_handle, SQLUSMALLINT column_number, SQLUSMALLINT field_identifier, + SQLPOINTER out_string_value, SQLSMALLINT out_string_value_max_size, SQLSMALLINT * out_string_value_size, + SQLLEN * out_num_value) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + Statement & statement = *reinterpret_cast(statement_handle); + + if (column_number < 1 || column_number > statement.columns_info.size()) + return SQL_ERROR; + + size_t column_idx = column_number - 1; + + SQLLEN num_value = 0; + std::string str_value; + + switch (field_identifier) + { + case SQL_DESC_AUTO_UNIQUE_VALUE: + break; + case SQL_DESC_BASE_COLUMN_NAME: + break; + case SQL_DESC_BASE_TABLE_NAME: + break; + case SQL_DESC_CASE_SENSITIVE: + num_value = SQL_TRUE; + break; + case SQL_DESC_CATALOG_NAME: + break; + case SQL_DESC_CONCISE_TYPE: + // TODO + break; + case SQL_DESC_COUNT: + num_value = statement.columns_info.size(); + break; + case SQL_DESC_DISPLAY_SIZE: + num_value = 0; //statement.connection.environment.types_info.at(statement.columns_info[column_idx].type).display_size; + break; + case SQL_DESC_FIXED_PREC_SCALE: + break; + case SQL_DESC_LABEL: + str_value = statement.columns_info[column_idx].name; + break; + case SQL_DESC_LENGTH: + break; + case SQL_DESC_LITERAL_PREFIX: + break; + case SQL_DESC_LITERAL_SUFFIX: + break; + case SQL_DESC_LOCAL_TYPE_NAME: + break; + case SQL_DESC_NAME: + str_value = statement.columns_info[column_idx].name; + break; + case SQL_DESC_NULLABLE: + num_value = SQL_FALSE; + break; + case SQL_DESC_NUM_PREC_RADIX: + break; + case SQL_DESC_OCTET_LENGTH: + break; + case SQL_DESC_PRECISION: + break; + case SQL_DESC_SCALE: + break; + case SQL_DESC_SCHEMA_NAME: + break; + case SQL_DESC_SEARCHABLE: + break; + case SQL_DESC_TABLE_NAME: + break; + case SQL_DESC_TYPE: + break; + case SQL_DESC_TYPE_NAME: + break; + case SQL_DESC_UNNAMED: + num_value = SQL_NAMED; + break; + case SQL_DESC_UNSIGNED: + num_value = statement.connection.environment.types_info.at(statement.columns_info[column_idx].type).is_unsigned; + break; + case SQL_DESC_UPDATABLE: + num_value = SQL_FALSE; + break; + default: + return SQL_ERROR; + } + + if (out_num_value) + *out_num_value = num_value; + + return fillOutputString(str_value.data(), str_value.size(), out_string_value, out_string_value_max_size, out_string_value_size); +} + + +RETCODE SQL_API +SQLFetch(HSTMT statement_handle) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + Statement & statement = *reinterpret_cast(statement_handle); + + bool res = statement.fetchRow(); + + return res ? SQL_SUCCESS : SQL_NO_DATA; +} + + +RETCODE SQL_API +SQLGetData(HSTMT statement_handle, + SQLUSMALLINT column_or_param_number, SQLSMALLINT target_type, + PTR out_value, SQLLEN out_value_max_size, + SQLLEN * out_value_size_or_indicator) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + Statement & statement = *reinterpret_cast(statement_handle); + + if (column_or_param_number < 1 || column_or_param_number > statement.columns_info.size()) + return SQL_ERROR; + + size_t column_idx = column_or_param_number - 1; + + std::cerr << "column: " << column_idx << ", target_type: " << target_type << "\n"; + + const std::string & value = statement.current_row[column_idx]; + + union + { + char bytes[8]; + uint64_t uint_data; + int64_t int_data; + float float_data; + double double_data; + } num; + size_t num_size = 0; + + switch (target_type) + { + case SQL_ARD_TYPE: + case SQL_C_DEFAULT: + return SQL_ERROR; + + case SQL_C_WCHAR: + case SQL_C_CHAR: + break; + + case SQL_C_TINYINT: + case SQL_C_STINYINT: + num_size = 1; + num.int_data = Statement::getInt(value); + break; + case SQL_C_UTINYINT: + num_size = 1; + num.uint_data = Statement::getUInt(value); + break; + + case SQL_C_SHORT: + case SQL_C_SSHORT: + num_size = 2; + num.int_data = Statement::getInt(value); + break; + case SQL_C_USHORT: + num_size = 2; + num.uint_data = Statement::getUInt(value); + break; + + case SQL_C_LONG: + case SQL_C_SLONG: + num_size = 4; + num.int_data = Statement::getInt(value); + break; + case SQL_C_ULONG: + num_size = 4; + num.uint_data = Statement::getUInt(value); + break; + + case SQL_C_SBIGINT: + num_size = 8; + num.int_data = Statement::getInt(value); + break; + case SQL_C_UBIGINT: + num_size = 8; + num.uint_data = Statement::getUInt(value); + break; + + case SQL_C_FLOAT: + num_size = 4; + num.float_data = Statement::getFloat(value); + break; + + case SQL_C_DOUBLE: + num_size = 8; + num.double_data = Statement::getDouble(value); + break; + + default: + return SQL_ERROR; + } + + if (num_size) + { + if (out_value_max_size < static_cast(num_size)) + return SQL_ERROR; + + memcpy(out_value, num.bytes, num_size); + + if (out_value_size_or_indicator) + *out_value_size_or_indicator = num_size; + } + else + { + if (target_type == SQL_C_CHAR) + { + return fillOutputString(value.data(), value.size(), out_value, out_value_max_size, out_value_size_or_indicator); + } + else + { + std::string converted; + //statement.connection.environment.converter_utf8_to_utf16.convert(value.data(), converted); + + converted.resize(value.size() * 2 + 1, '\xFF'); + converted[value.size() * 2] = '\0'; + for (size_t i = 0, size = value.size(); i < size; ++i) + converted[i * 2] = value[i]; + + return fillOutputString(converted.data(), converted.size(), out_value, out_value_max_size, out_value_size_or_indicator); + } + } + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLRowCount(HSTMT statement_handle, + SQLLEN * out_row_count) +{ + mylog(__FUNCTION__); + + if (nullptr == statement_handle) + return SQL_INVALID_HANDLE; + + Statement & statement = *reinterpret_cast(statement_handle); + + if (out_row_count) + *out_row_count = statement.row_count; + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLMoreResults(HSTMT hstmt) +{ + mylog(__FUNCTION__); + + return SQL_NO_DATA; +} + + +RETCODE SQL_API +SQLDisconnect(HDBC connection_handle) +{ + mylog(__FUNCTION__); + + if (nullptr == connection_handle) + return SQL_INVALID_HANDLE; + + Connection & connection = *reinterpret_cast(connection_handle); + + connection.session.reset(); + + return SQL_SUCCESS; +} + + +RETCODE SQL_API +SQLSetEnvAttr(SQLHENV environment_handle, SQLINTEGER attribute, + SQLPOINTER value, SQLINTEGER value_length) +{ + mylog(__FUNCTION__); + + if (nullptr == environment_handle) + return SQL_INVALID_HANDLE; + + Environment & environment = *reinterpret_cast(environment_handle); + + std::cerr << "attr: " << attribute << "\n"; + + switch (attribute) + { + case SQL_ATTR_CONNECTION_POOLING: + case SQL_ATTR_CP_MATCH: + case SQL_ATTR_OUTPUT_NTS: + default: + return SQL_ERROR; + + case SQL_ATTR_ODBC_VERSION: + intptr_t int_value = reinterpret_cast(value); + if (int_value != SQL_OV_ODBC2 && int_value != SQL_OV_ODBC3) + return SQL_ERROR; + + environment.odbc_version = int_value; + std::cerr << "Set ODBC version to " << int_value << "\n"; + + return SQL_SUCCESS; + } +} + + +RETCODE SQL_API +SQLGetEnvAttr(SQLHENV environment_handle, SQLINTEGER attribute, + SQLPOINTER out_value, SQLINTEGER out_value_max_length, SQLINTEGER * out_value_length) +{ + mylog(__FUNCTION__); + + if (nullptr == environment_handle) + return SQL_INVALID_HANDLE; + + Environment & environment = *reinterpret_cast(environment_handle); + + std::cerr << "attr: " << attribute << "\n"; + + switch (attribute) + { + case SQL_ATTR_CONNECTION_POOLING: + case SQL_ATTR_CP_MATCH: + case SQL_ATTR_OUTPUT_NTS: + default: + return SQL_ERROR; + + case SQL_ATTR_ODBC_VERSION: + *reinterpret_cast(out_value) = environment.odbc_version; + if (out_value_length) + *out_value_length = sizeof(SQLUINTEGER); + + return SQL_SUCCESS; + } + + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetConnectAttr(SQLHDBC connection_handle, SQLINTEGER attribute, + SQLPOINTER value, SQLINTEGER value_length) +{ + mylog(__FUNCTION__); + + if (nullptr == connection_handle) + return SQL_INVALID_HANDLE; + + std::cerr << "attr: " << attribute << "\n"; + + switch (attribute) + { + case SQL_ATTR_ACCESS_MODE: + case SQL_ATTR_ASYNC_ENABLE: + case SQL_ATTR_AUTO_IPD: + case SQL_ATTR_AUTOCOMMIT: + case SQL_ATTR_CONNECTION_DEAD: + case SQL_ATTR_CONNECTION_TIMEOUT: + case SQL_ATTR_CURRENT_CATALOG: + case SQL_ATTR_LOGIN_TIMEOUT: /// TODO + case SQL_ATTR_METADATA_ID: + case SQL_ATTR_ODBC_CURSORS: + case SQL_ATTR_PACKET_SIZE: + case SQL_ATTR_QUIET_MODE: + case SQL_ATTR_TRACE: + case SQL_ATTR_TRACEFILE: + case SQL_ATTR_TRANSLATE_LIB: + case SQL_ATTR_TRANSLATE_OPTION: + case SQL_ATTR_TXN_ISOLATION: + default: + return SQL_ERROR; + } +} + + +RETCODE SQL_API +SQLGetDiagRec(SQLSMALLINT handle_type, SQLHANDLE handle, + SQLSMALLINT record_number, + SQLCHAR * out_sqlstate, + SQLINTEGER * out_native_error_code, + SQLCHAR * out_mesage, SQLSMALLINT out_message_max_size, SQLSMALLINT * out_message_size) +{ + mylog(__FUNCTION__); + + std::cerr << "handle_type: " << handle_type << ", record_number: " << record_number << ", out_message_max_size: " << out_message_max_size << "\n"; + + if (nullptr == handle) + return SQL_INVALID_HANDLE; + + if (record_number <= 0 || out_message_max_size < 0) + return SQL_ERROR; + + if (record_number > 1) + return SQL_NO_DATA; + + DiagnosticRecord * diagnostic_record = nullptr; + switch (handle_type) + { + case SQL_HANDLE_ENV: + diagnostic_record = &reinterpret_cast(handle)->diagnostic_record; + break; + case SQL_HANDLE_DBC: + diagnostic_record = &reinterpret_cast(handle)->diagnostic_record; + break; + case SQL_HANDLE_STMT: + diagnostic_record = &reinterpret_cast(handle)->diagnostic_record; + break; + default: + return SQL_ERROR; + } + + if (diagnostic_record->native_error_code == 0) + return SQL_NO_DATA; + + /// Пятибуквенный SQLSTATE и завершающий ноль. + if (out_sqlstate) + strncpy(reinterpret_cast(out_sqlstate), diagnostic_record->sql_state.data(), 6); + + if (out_native_error_code) + *out_native_error_code = diagnostic_record->native_error_code; + + return fillOutputString(diagnostic_record->message.data(), diagnostic_record->message.size(), out_mesage, out_message_max_size, out_message_size); +} + + +RETCODE SQL_API +SQLGetDiagField(SQLSMALLINT handle_type, SQLHANDLE handle, + SQLSMALLINT record_number, + SQLSMALLINT field_id, + SQLPOINTER out_mesage, SQLSMALLINT out_message_max_size, SQLSMALLINT * out_message_size) +{ + mylog(__FUNCTION__); + + return SQLGetDiagRec( + handle_type, + handle, + record_number, + nullptr, + nullptr, + reinterpret_cast(out_mesage), + out_message_max_size, + out_message_size); +} + + +RETCODE SQL_API +SQLTables(HSTMT StatementHandle, + SQLCHAR *CatalogName, SQLSMALLINT NameLength1, + SQLCHAR *SchemaName, SQLSMALLINT NameLength2, + SQLCHAR *TableName, SQLSMALLINT NameLength3, + SQLCHAR *TableType, SQLSMALLINT NameLength4) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLBrowseConnect(HDBC connection_handle, + SQLCHAR *szConnStrIn, + SQLSMALLINT cbConnStrIn, + SQLCHAR *szConnStrOut, + SQLSMALLINT cbConnStrOutMax, + SQLSMALLINT *pcbConnStrOut) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLBindCol(HSTMT StatementHandle, + SQLUSMALLINT ColumnNumber, SQLSMALLINT TargetType, + PTR TargetValue, SQLLEN BufferLength, + SQLLEN *StrLen_or_Ind) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLCancel(HSTMT StatementHandle) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLColumns(HSTMT StatementHandle, + SQLCHAR *CatalogName, SQLSMALLINT NameLength1, + SQLCHAR *SchemaName, SQLSMALLINT NameLength2, + SQLCHAR *TableName, SQLSMALLINT NameLength3, + SQLCHAR *ColumnName, SQLSMALLINT NameLength4) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLDataSources(HENV EnvironmentHandle, + SQLUSMALLINT Direction, SQLCHAR *ServerName, + SQLSMALLINT BufferLength1, SQLSMALLINT *NameLength1, + SQLCHAR *Description, SQLSMALLINT BufferLength2, + SQLSMALLINT *NameLength2) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLDescribeCol(HSTMT StatementHandle, + SQLUSMALLINT ColumnNumber, SQLCHAR *ColumnName, + SQLSMALLINT BufferLength, SQLSMALLINT *NameLength, + SQLSMALLINT *DataType, SQLULEN *ColumnSize, + SQLSMALLINT *DecimalDigits, SQLSMALLINT *Nullable) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetCursorName(HSTMT StatementHandle, + SQLCHAR *CursorName, SQLSMALLINT BufferLength, + SQLSMALLINT *NameLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +/* +/// Эта функция может быть реализована в driver manager-е. +RETCODE SQL_API +SQLGetFunctions(HDBC ConnectionHandle, + SQLUSMALLINT FunctionId, SQLUSMALLINT *Supported) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +}*/ + + +RETCODE SQL_API +SQLGetTypeInfo(HSTMT StatementHandle, + SQLSMALLINT DataType) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLParamData(HSTMT StatementHandle, + PTR *Value) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLPutData(HSTMT StatementHandle, + PTR Data, SQLLEN StrLen_or_Ind) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetCursorName(HSTMT StatementHandle, + SQLCHAR *CursorName, SQLSMALLINT NameLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetParam(HSTMT StatementHandle, + SQLUSMALLINT ParameterNumber, SQLSMALLINT ValueType, + SQLSMALLINT ParameterType, SQLULEN LengthPrecision, + SQLSMALLINT ParameterScale, PTR ParameterValue, + SQLLEN *StrLen_or_Ind) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSpecialColumns(HSTMT StatementHandle, + SQLUSMALLINT IdentifierType, SQLCHAR *CatalogName, + SQLSMALLINT NameLength1, SQLCHAR *SchemaName, + SQLSMALLINT NameLength2, SQLCHAR *TableName, + SQLSMALLINT NameLength3, SQLUSMALLINT Scope, + SQLUSMALLINT Nullable) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLStatistics(HSTMT StatementHandle, + SQLCHAR *CatalogName, SQLSMALLINT NameLength1, + SQLCHAR *SchemaName, SQLSMALLINT NameLength2, + SQLCHAR *TableName, SQLSMALLINT NameLength3, + SQLUSMALLINT Unique, SQLUSMALLINT Reserved) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLColumnPrivileges(HSTMT hstmt, + SQLCHAR *szCatalogName, + SQLSMALLINT cbCatalogName, + SQLCHAR *szSchemaName, + SQLSMALLINT cbSchemaName, + SQLCHAR *szTableName, + SQLSMALLINT cbTableName, + SQLCHAR *szColumnName, + SQLSMALLINT cbColumnName) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLDescribeParam(HSTMT hstmt, + SQLUSMALLINT ipar, + SQLSMALLINT *pfSqlType, + SQLULEN *pcbParamDef, + SQLSMALLINT *pibScale, + SQLSMALLINT *pfNullable) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLExtendedFetch(HSTMT hstmt, + SQLUSMALLINT fFetchType, + SQLLEN irow, +#if defined(WITH_UNIXODBC) && (SIZEOF_LONG != 8) + SQLROWSETSIZE *pcrow, +#else + SQLULEN *pcrow, +#endif /* WITH_UNIXODBC */ + SQLUSMALLINT *rgfRowStatus) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLForeignKeys(HSTMT hstmt, + SQLCHAR *szPkCatalogName, + SQLSMALLINT cbPkCatalogName, + SQLCHAR *szPkSchemaName, + SQLSMALLINT cbPkSchemaName, + SQLCHAR *szPkTableName, + SQLSMALLINT cbPkTableName, + SQLCHAR *szFkCatalogName, + SQLSMALLINT cbFkCatalogName, + SQLCHAR *szFkSchemaName, + SQLSMALLINT cbFkSchemaName, + SQLCHAR *szFkTableName, + SQLSMALLINT cbFkTableName) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLNativeSql(HDBC hdbc, + SQLCHAR *szSqlStrIn, + SQLINTEGER cbSqlStrIn, + SQLCHAR *szSqlStr, + SQLINTEGER cbSqlStrMax, + SQLINTEGER *pcbSqlStr) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLNumParams(HSTMT hstmt, + SQLSMALLINT *pcpar) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLPrimaryKeys(HSTMT hstmt, + SQLCHAR *szCatalogName, + SQLSMALLINT cbCatalogName, + SQLCHAR *szSchemaName, + SQLSMALLINT cbSchemaName, + SQLCHAR *szTableName, + SQLSMALLINT cbTableName) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLProcedureColumns(HSTMT hstmt, + SQLCHAR *szCatalogName, + SQLSMALLINT cbCatalogName, + SQLCHAR *szSchemaName, + SQLSMALLINT cbSchemaName, + SQLCHAR *szProcName, + SQLSMALLINT cbProcName, + SQLCHAR *szColumnName, + SQLSMALLINT cbColumnName) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLProcedures(HSTMT hstmt, + SQLCHAR *szCatalogName, + SQLSMALLINT cbCatalogName, + SQLCHAR *szSchemaName, + SQLSMALLINT cbSchemaName, + SQLCHAR *szProcName, + SQLSMALLINT cbProcName) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetPos(HSTMT hstmt, + SQLSETPOSIROW irow, + SQLUSMALLINT fOption, + SQLUSMALLINT fLock) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLTablePrivileges(HSTMT hstmt, + SQLCHAR *szCatalogName, + SQLSMALLINT cbCatalogName, + SQLCHAR *szSchemaName, + SQLSMALLINT cbSchemaName, + SQLCHAR *szTableName, + SQLSMALLINT cbTableName) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLBindParameter(HSTMT hstmt, + SQLUSMALLINT ipar, + SQLSMALLINT fParamType, + SQLSMALLINT fCType, + SQLSMALLINT fSqlType, + SQLULEN cbColDef, + SQLSMALLINT ibScale, + PTR rgbValue, + SQLLEN cbValueMax, + SQLLEN *pcbValue) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + +/* +RETCODE SQL_API +SQLBulkOperations( + SQLHSTMT StatementHandle, + SQLUSMALLINT Operation) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +}*/ + + +RETCODE SQL_API +SQLCancelHandle( + SQLSMALLINT HandleType, + SQLHANDLE Handle) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLCloseCursor( + SQLHSTMT StatementHandle) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLCompleteAsync( + SQLSMALLINT HandleType, + SQLHANDLE Handle, + RETCODE * AsyncRetCodePtr) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLCopyDesc( + SQLHDESC SourceDescHandle, + SQLHDESC TargetDescHandle) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLEndTran( + SQLSMALLINT HandleType, + SQLHANDLE Handle, + SQLSMALLINT CompletionType) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLError(SQLHENV hDrvEnv, SQLHDBC hDrvDbc, SQLHSTMT hDrvStmt, + SQLCHAR *szSqlState, SQLINTEGER *pfNativeError, SQLCHAR *szErrorMsg, + SQLSMALLINT nErrorMsgMax, SQLSMALLINT *pcbErrorMsg) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLFetchScroll(SQLHSTMT hDrvStmt, SQLSMALLINT nOrientation, SQLLEN nOffset) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetConnectAttr(SQLHDBC hDrvDbc, SQLINTEGER Attribute, SQLPOINTER Value, + SQLINTEGER BufferLength, SQLINTEGER *StringLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetConnectOption(SQLHDBC hDrvDbc, UWORD fOption, PTR pvParam) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetDescField(SQLHDESC DescriptorHandle, SQLSMALLINT RecordNumber, + SQLSMALLINT FieldIdentifier, SQLPOINTER Value, SQLINTEGER BufferLength, + SQLINTEGER *StringLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetDescRec(SQLHDESC DescriptorHandle, SQLSMALLINT RecordNumber, + SQLCHAR *Name, SQLSMALLINT BufferLength, SQLSMALLINT *StringLength, + SQLSMALLINT *Type, SQLSMALLINT *SubType, SQLLEN *Length, + SQLSMALLINT *Precision, SQLSMALLINT *Scale, SQLSMALLINT *Nullable) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetStmtAttr(SQLHSTMT hDrvStmt, SQLINTEGER Attribute, SQLPOINTER Value, + SQLINTEGER BufferLength, SQLINTEGER *StringLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLGetStmtOption(SQLHSTMT hDrvStmt, UWORD fOption, PTR pvParam) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLParamOptions(SQLHSTMT hDrvStmt, SQLULEN nRow, SQLULEN *pnRow) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetConnectOption(SQLHDBC hDrvDbc, UWORD nOption, SQLULEN vParam) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetDescField(SQLHDESC DescriptorHandle, SQLSMALLINT RecordNumber, + SQLSMALLINT FieldIdentifier, SQLPOINTER Value, SQLINTEGER BufferLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetDescRec(SQLHDESC hDescriptorHandle, SQLSMALLINT nRecordNumber, + SQLSMALLINT nType, SQLSMALLINT nSubType, SQLLEN nLength, + SQLSMALLINT nPrecision, SQLSMALLINT nScale, SQLPOINTER pData, + SQLLEN *pnStringLength, SQLLEN *pnIndicator) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetScrollOptions( + SQLHSTMT hDrvStmt, SQLUSMALLINT fConcurrency, SQLLEN crowKeyset, + SQLUSMALLINT crowRowset) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetStmtAttr(SQLHSTMT hDrvStmt, SQLINTEGER Attribute, SQLPOINTER Value, + SQLINTEGER StringLength) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLSetStmtOption(SQLHSTMT hDrvStmt, UWORD fOption, SQLULEN vParam) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +RETCODE SQL_API +SQLTransact(SQLHENV hDrvEnv, SQLHDBC hDrvDbc, UWORD nType) +{ + mylog(__FUNCTION__); + return SQL_ERROR; +} + + +} diff --git a/dbms/src/Parsers/ParserCheckQuery.cpp b/dbms/src/Parsers/ParserCheckQuery.cpp index d18b307f746..96a9ba66b03 100644 --- a/dbms/src/Parsers/ParserCheckQuery.cpp +++ b/dbms/src/Parsers/ParserCheckQuery.cpp @@ -8,10 +8,8 @@ using namespace DB; bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) { - ParserWhiteSpaceOrComments ws; ParserString s_check("CHECK", true, true); ParserString s_table("TABLE", true, true); - ParserString s_format("FORMAT", true, true); ParserString s_dot("."); ParserIdentifier table_parser; @@ -50,18 +48,8 @@ bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr & ws.ignore(pos, end); /// FORMAT format_name - if (s_format.ignore(pos, end, max_parsed_pos, expected)) - { - ws.ignore(pos, end); - - ParserIdentifier format_p; - - if (!format_p.parse(pos, end, query->format, max_parsed_pos, expected)) - return false; - typeid_cast(*query->format).kind = ASTIdentifier::Format; - - ws.ignore(pos, end); - } + if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected)) + return false; node = query; return true; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp new file mode 100644 index 00000000000..ea0ec25aeaa --- /dev/null +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ + +bool ParserQueryWithOutput::parseFormat(ASTQueryWithOutput & query, Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) +{ + ParserString s_format("FORMAT", true, true); + + if (s_format.ignore(pos, end, max_parsed_pos, expected)) + { + ws.ignore(pos, end); + + ParserIdentifier format_p; + + if (!format_p.parse(pos, end, query.format, max_parsed_pos, expected)) + return false; + typeid_cast(*(query.format)).kind = ASTIdentifier::Format; + + ws.ignore(pos, end); + } + + return true; +} + +} diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index ae2b3084d4c..393687db555 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -19,7 +19,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p ASTSelectQuery * select_query = new ASTSelectQuery; node = select_query; - ParserWhiteSpaceOrComments ws; ParserString s_select("SELECT", true, true); ParserString s_distinct("DISTINCT", true, true); ParserString s_from("FROM", true, true); @@ -39,7 +38,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p ParserString s_order("ORDER", true, true); ParserString s_limit("LIMIT", true, true); ParserString s_settings("SETTINGS", true, true); - ParserString s_format("FORMAT", true, true); ParserString s_union("UNION", true, true); ParserString s_all("ALL", true, true); @@ -310,22 +308,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p ws.ignore(pos, end); } - bool has_format = false; - /// FORMAT format_name - if (s_format.ignore(pos, end, max_parsed_pos, expected)) - { - ws.ignore(pos, end); - - ParserIdentifier format_p; - - if (!format_p.parse(pos, end, select_query->format, max_parsed_pos, expected)) - return false; - typeid_cast(*select_query->format).kind = ASTIdentifier::Format; - - ws.ignore(pos, end); - has_format = true; - } + if (!parseFormat(*select_query, pos, end, node, max_parsed_pos, expected)) + return false; // UNION ALL select query if (s_union.ignore(pos, end, max_parsed_pos, expected)) @@ -334,7 +319,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p if (s_all.ignore(pos, end, max_parsed_pos, expected)) { - if (has_format) + if (!select_query->format.isNull()) { /// FORMAT может быть задан только в последнем запросе цепочки UNION ALL. expected = "FORMAT only in the last SELECT of the UNION ALL chain"; diff --git a/dbms/src/Parsers/ParserShowTablesQuery.cpp b/dbms/src/Parsers/ParserShowTablesQuery.cpp index 1a0af9cebdd..0c0d12b0756 100644 --- a/dbms/src/Parsers/ParserShowTablesQuery.cpp +++ b/dbms/src/Parsers/ParserShowTablesQuery.cpp @@ -15,20 +15,17 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m { Pos begin = pos; - ParserWhiteSpaceOrComments ws; ParserString s_show("SHOW", true, true); ParserString s_tables("TABLES", true, true); ParserString s_databases("DATABASES", true, true); ParserString s_from("FROM", true, true); ParserString s_not("NOT", true, true); ParserString s_like("LIKE", true, true); - ParserString s_format("FORMAT", true, true); ParserStringLiteral like_p; ParserIdentifier name_p; ASTPtr like; ASTPtr database; - ASTPtr format; ASTShowTablesQuery * query = new ASTShowTablesQuery; ASTPtr query_ptr = query; @@ -80,18 +77,9 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m ws.ignore(pos, end); - if (s_format.ignore(pos, end, max_parsed_pos, expected)) - { - ws.ignore(pos, end); - - ParserIdentifier format_p; - - if (!format_p.parse(pos, end, format, max_parsed_pos, expected)) - return false; - typeid_cast(*format).kind = ASTIdentifier::Format; - - ws.ignore(pos, end); - } + /// FORMAT format_name + if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected)) + return false; query->range = StringRange(begin, pos); @@ -99,11 +87,8 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m query->from = typeid_cast(*database).name; if (like) query->like = safeGet(typeid_cast(*like).value); - if (format) - { - query->format = format; - query->children.push_back(format); - } + if (query->format) + query->children.push_back(query->format); node = query_ptr; diff --git a/dbms/src/Parsers/ParserTablePropertiesQuery.cpp b/dbms/src/Parsers/ParserTablePropertiesQuery.cpp index 6f0b1ef067c..ae374a0822d 100644 --- a/dbms/src/Parsers/ParserTablePropertiesQuery.cpp +++ b/dbms/src/Parsers/ParserTablePropertiesQuery.cpp @@ -13,20 +13,17 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po { Pos begin = pos; - ParserWhiteSpaceOrComments ws; ParserString s_exists("EXISTS", true, true); ParserString s_describe("DESCRIBE", true, true); ParserString s_desc("DESC", true, true); ParserString s_show("SHOW", true, true); ParserString s_create("CREATE", true, true); ParserString s_table("TABLE", true, true); - ParserString s_format("FORMAT", true, true); ParserString s_dot("."); ParserIdentifier name_p; ASTPtr database; ASTPtr table; - ASTPtr format; ASTPtr query_ptr; ws.ignore(pos, end); @@ -53,6 +50,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po return false; } + ASTQueryWithTableAndOutput * query = dynamic_cast(&*query_ptr); ws.ignore(pos, end); @@ -76,20 +74,9 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po ws.ignore(pos, end); - if (s_format.ignore(pos, end, max_parsed_pos, expected)) - { - ws.ignore(pos, end); - - ParserIdentifier format_p; - - if (!format_p.parse(pos, end, format, max_parsed_pos, expected)) - return false; - typeid_cast(*format).kind = ASTIdentifier::Format; - - ws.ignore(pos, end); - } - - ASTQueryWithTableAndOutput * query = dynamic_cast(&*query_ptr); + /// FORMAT format_name + if (!parseFormat(*query, pos, end, node, max_parsed_pos, expected)) + return false; query->range = StringRange(begin, pos); @@ -97,11 +84,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po query->database = typeid_cast(*database).name; if (table) query->table = typeid_cast(*table).name; - if (format) - { - query->format = format; - query->children.push_back(format); - } + if (query->format) + query->children.push_back(query->format); node = query_ptr; diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 37ee684d646..8da9e911c86 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -8,12 +8,12 @@ #include #include + namespace DB { - void AlterCommand::apply(NamesAndTypesList & columns, - NamesAndTypesList & materialized_columns, - NamesAndTypesList & alias_columns, - ColumnDefaults & column_defaults) const + void AlterCommand::apply( + NamesAndTypesList & columns, NamesAndTypesList & materialized_columns, NamesAndTypesList & alias_columns, + ColumnDefaults & column_defaults) const { if (type == ADD) { @@ -27,7 +27,7 @@ namespace DB exists_in(alias_columns)) { throw Exception{ - "Cannot add column " + column_name + ": column with this name already exisits.", + "Cannot add column " + column_name + ": column with this name already exists", DB::ErrorCodes::ILLEGAL_COLUMN }; } @@ -99,40 +99,37 @@ namespace DB } else if (type == MODIFY) { - const auto it = column_defaults.find(column_name); - const auto had_default_expr = it != column_defaults.end(); - const auto old_default_type = had_default_expr ? it->second.type : ColumnDefaultType{}; + const auto default_it = column_defaults.find(column_name); + const auto had_default_expr = default_it != std::end(column_defaults); + const auto old_default_type = had_default_expr ? default_it->second.type : ColumnDefaultType{}; - /// allow conversion between DEFAULT and MATERIALIZED - const auto default_materialized_conversion = - (old_default_type == ColumnDefaultType::Default && default_type == ColumnDefaultType::Materialized) || - (old_default_type == ColumnDefaultType::Materialized && default_type == ColumnDefaultType::Default); - - if (old_default_type != default_type && !default_materialized_conversion) - throw Exception{"Cannot change column default specifier from " + toString(old_default_type) + - " to " + toString(default_type), ErrorCodes::INCORRECT_QUERY}; + /// target column list + auto & new_columns = default_type == ColumnDefaultType::Default ? + columns : default_type == ColumnDefaultType::Materialized ? + materialized_columns : alias_columns; /// find column or throw exception const auto find_column = [this] (NamesAndTypesList & columns) { const auto it = std::find_if(columns.begin(), columns.end(), std::bind(namesEqual, std::cref(column_name), std::placeholders::_1) ); if (it == columns.end()) - throw Exception("Wrong column name. Cannot find column " + column_name + " to modify.", + throw Exception("Wrong column name. Cannot find column " + column_name + " to modify", DB::ErrorCodes::ILLEGAL_COLUMN); return it; }; - /// remove from the old list, add to the new list in case of DEFAULT <-> MATERIALIZED alteration - if (default_materialized_conversion) + /// if default types differ, remove column from the old list, then add to the new list + if (default_type != old_default_type) { - const auto was_default = old_default_type == ColumnDefaultType::Default; - auto & old_columns = was_default ? columns : materialized_columns; - auto & new_columns = was_default ? materialized_columns : columns; + /// source column list + auto & old_columns = old_default_type == ColumnDefaultType::Default ? + columns : old_default_type == ColumnDefaultType::Materialized ? + materialized_columns : alias_columns; - const auto column_it = find_column(old_columns); - new_columns.emplace_back(*column_it); - old_columns.erase(column_it); + const auto old_column_it = find_column(old_columns); + new_columns.emplace_back(*old_column_it); + old_columns.erase(old_column_it); /// do not forget to change the default type of old column if (had_default_expr) @@ -140,19 +137,17 @@ namespace DB } /// find column in one of three column lists - const auto column_it = find_column( - default_type == ColumnDefaultType::Default ? columns : - default_type == ColumnDefaultType::Materialized ? materialized_columns : - alias_columns); - + const auto column_it = find_column(new_columns); column_it->type = data_type; - /// remove, add or update default_expression if (!default_expression && had_default_expr) + /// new column has no default expression, remove it from column_defaults along with it's type column_defaults.erase(column_name); else if (default_expression && !had_default_expr) + /// new column has a default expression while the old one had not, add it it column_defaults column_defaults.emplace(column_name, ColumnDefault{default_type, default_expression}); else if (had_default_expr) + /// both old and new columns have default expression, update it column_defaults[column_name].expression = default_expression; } else @@ -184,7 +179,7 @@ namespace DB columns.insert(std::end(columns), std::begin(table->alias_columns), std::end(table->alias_columns)); auto defaults = table->column_defaults; - std::vector> defaulted_columns{}; + std::vector> defaulted_columns{}; ASTPtr default_expr_list{new ASTExpressionList}; default_expr_list->children.reserve(defaults.size()); @@ -193,32 +188,44 @@ namespace DB { if (command.type == AlterCommand::ADD || command.type == AlterCommand::MODIFY) { - if (command.type == AlterCommand::MODIFY) + const auto & column_name = command.column_name; + const auto column_it = std::find_if(std::begin(columns), std::end(columns), + std::bind(AlterCommand::namesEqual, std::cref(command.column_name), std::placeholders::_1)); + + if (command.type == AlterCommand::ADD) + { + if (std::end(columns) != column_it) + throw Exception{ + "Cannot add column " + column_name + ": column with this name already exisits", + DB::ErrorCodes::ILLEGAL_COLUMN + }; + } + else if (command.type == AlterCommand::MODIFY) { - const auto it = std::find_if(std::begin(columns), std::end(columns), - std::bind(AlterCommand::namesEqual, std::cref(command.column_name), std::placeholders::_1)); - if (it == std::end(columns)) - throw Exception("Wrong column name. Cannot find column " + command.column_name + " to modify.", - DB::ErrorCodes::ILLEGAL_COLUMN); + if (std::end(columns) == column_it) + throw Exception{ + "Wrong column name. Cannot find column " + column_name + " to modify", + DB::ErrorCodes::ILLEGAL_COLUMN + }; - columns.erase(it); - defaults.erase(command.column_name); + columns.erase(column_it); + defaults.erase(column_name); } /// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions - columns.emplace_back(command.column_name, command.data_type ? command.data_type : new DataTypeUInt8); + columns.emplace_back(column_name, command.data_type ? command.data_type : new DataTypeUInt8); if (command.default_expression) { if (command.data_type) { - const auto & final_column_name = command.column_name; + const auto & final_column_name = column_name; const auto tmp_column_name = final_column_name + "_tmp"; - const auto data_type_ptr = command.data_type.get(); + const auto column_type_raw_ptr = command.data_type.get(); /// specific code for different data types, e.g. toFixedString(col, N) for DataTypeFixedString - if (const auto fixed_string = typeid_cast(data_type_ptr)) + if (const auto fixed_string = typeid_cast(column_type_raw_ptr)) { const auto conversion_function_name = "toFixedString"; @@ -229,10 +236,15 @@ namespace DB ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}), final_column_name)); } + else if (typeid_cast(column_type_raw_ptr)) + { + /// do not perform conversion on arrays, require exact type match + default_expr_list->children.emplace_back(setAlias( + command.default_expression->clone(), final_column_name)); + } else { - /// @todo fix for parametric types, results in broken codem, i.e. toArray(ElementType)(col) - const auto conversion_function_name = "to" + data_type_ptr->getName(); + const auto conversion_function_name = "to" + column_type_raw_ptr->getName(); default_expr_list->children.emplace_back(setAlias( makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}), @@ -241,14 +253,15 @@ namespace DB default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name)); - defaulted_columns.emplace_back(command.column_name, &command); + defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command); } else { + /// no type explicitly specified, will deduce later default_expr_list->children.emplace_back( - setAlias(command.default_expression->clone(), command.column_name)); + setAlias(command.default_expression->clone(), column_name)); - defaulted_columns.emplace_back(command.column_name, &command); + defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command); } } } @@ -271,7 +284,7 @@ namespace DB ++it; if (!found) - throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop.", + throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop", DB::ErrorCodes::ILLEGAL_COLUMN); } } @@ -285,15 +298,38 @@ namespace DB return AlterCommand::namesEqual(column_name, name_type); }); const auto tmp_column_name = column_name + "_tmp"; - const auto conversion_function_name = "to" + column_it->type->getName(); + const auto & column_type_ptr = column_it->type; + const auto column_type_raw_ptr = column_type_ptr.get(); - default_expr_list->children.emplace_back(setAlias( - makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}), - column_name)); + /// specific code for different data types, e.g. toFixedString(col, N) for DataTypeFixedString + if (const auto fixed_string = typeid_cast(column_type_raw_ptr)) + { + default_expr_list->children.emplace_back(setAlias( + makeASTFunction("toFixedString", + ASTPtr{new ASTIdentifier{{}, tmp_column_name}}, + ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}), + column_name)); + } + else if (typeid_cast(column_type_raw_ptr)) + { + /// do not perform conversion on arrays, require exact type match + default_expr_list->children.emplace_back(setAlias( + col_def.second.expression->clone(), + column_name)); + } + else + { + const auto conversion_function_name = "to" + column_it->type->getName(); + + default_expr_list->children.emplace_back(setAlias( + makeASTFunction(conversion_function_name, + ASTPtr{new ASTIdentifier{{}, tmp_column_name}}), + column_name)); + } default_expr_list->children.emplace_back(setAlias(col_def.second.expression->clone(), tmp_column_name)); - defaulted_columns.emplace_back(column_name, nullptr); + defaulted_columns.emplace_back(NameAndTypePair{column_name, column_type_ptr}, nullptr); } const auto actions = ExpressionAnalyzer{default_expr_list, context, {}, columns}.getActions(true); @@ -302,41 +338,62 @@ namespace DB /// set deduced types, modify default expression if necessary for (auto & defaulted_column : defaulted_columns) { - const auto & column_name = defaulted_column.first; - const auto command_ptr = defaulted_column.second; - const auto & column = block.getByName(column_name); + const auto & name_and_type = defaulted_column.first; + AlterCommand * & command_ptr = defaulted_column.second; + + const auto & column_name = name_and_type.name; + const auto has_explicit_type = nullptr != name_and_type.type; /// default expression on old column - if (!command_ptr) + if (has_explicit_type) { + const auto & explicit_type = name_and_type.type; const auto & tmp_column = block.getByName(column_name + "_tmp"); + const auto & deduced_type = tmp_column.type; // column not specified explicitly in the ALTER query may require default_expression modification - if (column.type->getName() != tmp_column.type->getName()) + if (explicit_type->getName() != deduced_type->getName()) { - const auto it = defaults.find(column_name); - this->push_back(AlterCommand{ - AlterCommand::MODIFY, column_name, column.type, it->second.type, - makeASTFunction("to" + column.type->getName(), it->second.expression), - }); - } - } - else if (command_ptr && command_ptr->data_type) - { - const auto & tmp_column = block.getByName(column_name + "_tmp"); + const auto default_it = defaults.find(column_name); - /// type mismatch between explicitly specified and deduced type, add conversion - if (column.type->getName() != tmp_column.type->getName()) - { - command_ptr->default_expression = makeASTFunction( - "to" + column.type->getName(), - command_ptr->default_expression->clone()); + /// column has no associated alter command, let's create it + if (!command_ptr) + { + /// add a new alter command to modify existing column + this->emplace_back(AlterCommand{ + AlterCommand::MODIFY, column_name, explicit_type, + default_it->second.type, default_it->second.expression + }); + + command_ptr = &this->back(); + } + + if (const auto fixed_string = typeid_cast(explicit_type.get())) + { + command_ptr->default_expression = makeASTFunction("toFixedString", + command_ptr->default_expression->clone(), + ASTPtr{new ASTLiteral{{}, fixed_string->getN()}}); + } + else if (typeid_cast(explicit_type.get())) + { + /// foolproof against defaulting array columns incorrectly + throw Exception{ + "Default expression type mismatch for column " + column_name + ". Expected " + + explicit_type->getName() + ", deduced " + deduced_type->getName(), + ErrorCodes::TYPE_MISMATCH + }; + } + else + { + command_ptr->default_expression = makeASTFunction("to" + explicit_type->getName(), + command_ptr->default_expression->clone()); + } } } else { /// just set deduced type - command_ptr->data_type = column.type; + command_ptr->data_type = block.getByName(column_name).type; } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 022588e2e8c..10fbd24234b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -557,6 +557,8 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( if (expression) { MarkRanges ranges(1, MarkRange(0, part->size)); + /** @todo expression->getRequiedColumns may contain integer width columns for FixedString(N) type which after + * passing them to ITableDeclaration::check will trigger and exception about unknown column `N` */ BlockInputStreamPtr part_in = new MergeTreeBlockInputStream(full_path + part->name + '/', DEFAULT_MERGE_BLOCK_SIZE, expression->getRequiredColumns(), *this, part, ranges, false, nullptr, "", false, 0, DBMS_DEFAULT_BUFFER_SIZE); ExpressionBlockInputStream in(part_in, expression); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 9de9b99ac7d..030a04dfae2 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -80,7 +80,7 @@ StorageDistributed::StorageDistributed( context(context_), cluster(cluster_), sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, *columns).getActions(false) : nullptr), sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}), - write_enabled(!data_path_.empty() && (cluster.getLocalNodesNum() + cluster.pools.size() < 2 || sharding_key_)), + write_enabled(!data_path_.empty() && (((cluster.getLocalShardCount() + cluster.getRemoteShardCount()) < 2) || sharding_key_)), path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/')) { createDirectoryMonitors(); @@ -104,7 +104,7 @@ StorageDistributed::StorageDistributed( context(context_), cluster(cluster_), sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, *columns).getActions(false) : nullptr), sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}), - write_enabled(!data_path_.empty() && (cluster.getLocalNodesNum() + cluster.pools.size() < 2 || sharding_key_)), + write_enabled(!data_path_.empty() && (((cluster.getLocalShardCount() + cluster.getRemoteShardCount()) < 2) || sharding_key_)), path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/')) { createDirectoryMonitors(); @@ -168,7 +168,7 @@ BlockInputStreams StorageDistributed::read( /// Не имеет смысла на удалённых серверах, так как запрос отправляется обычно с другим user-ом. new_settings.max_concurrent_queries_for_user = 0; - size_t result_size = (cluster.pools.size() * settings.max_parallel_replicas) + cluster.getLocalNodesNum(); + size_t result_size = (cluster.getRemoteShardCount() * settings.max_parallel_replicas) + cluster.getLocalShardCount(); processed_stage = result_size == 1 || settings.distributed_group_by_no_merge ? QueryProcessingStage::Complete @@ -193,26 +193,31 @@ BlockInputStreams StorageDistributed::read( external_tables = context.getExternalTables(); /// Цикл по шардам. - for (auto & conn_pool : cluster.pools) - res.emplace_back(new RemoteBlockInputStream{ - conn_pool, modified_query, &new_settings, throttler, - external_tables, processed_stage, context}); - - /// Добавляем запросы к локальному ClickHouse. - if (cluster.getLocalNodesNum() > 0) + for (const auto & shard_info : cluster.getShardsInfo()) { - DB::Context new_context = context; - new_context.setSettings(new_settings); - - for (size_t i = 0; i < cluster.getLocalNodesNum(); ++i) + if (shard_info.isLocal()) { - InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage); + /// Добавляем запросы к локальному ClickHouse. - /** Материализация нужна, так как с удалённых серверов константы приходят материализованными. - * Если этого не делать, то в разных потоках будут получаться разные типы (Const и не-Const) столбцов, - * а это не разрешено, так как весь код исходит из допущения, что в потоке блоков все типы одинаковые. - */ - res.emplace_back(new MaterializingBlockInputStream(interpreter.execute().in)); + DB::Context new_context = context; + new_context.setSettings(new_settings); + + for (const auto & address : shard_info.local_addresses) + { + InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage); + + /** Материализация нужна, так как с удалённых серверов константы приходят материализованными. + * Если этого не делать, то в разных потоках будут получаться разные типы (Const и не-Const) столбцов, + * а это не разрешено, так как весь код исходит из допущения, что в потоке блоков все типы одинаковые. + */ + res.emplace_back(new MaterializingBlockInputStream(interpreter.execute().in)); + } + } + else + { + res.emplace_back(new RemoteBlockInputStream{ + shard_info.pool, modified_query, &new_settings, throttler, + external_tables, processed_stage, context}); } } @@ -274,26 +279,29 @@ BlockInputStreams StorageDistributed::describe(const Context & context, const Se BlockInputStreams res; /// Цикл по шардам. - for (auto & conn_pool : cluster.pools) + for (const auto & shard_info : cluster.getShardsInfo()) { - auto stream = new RemoteBlockInputStream{conn_pool, query, &new_settings, throttler}; - stream->reachAllReplicas(); - stream->appendExtraInfo(); - res.emplace_back(stream); - } - - /// Добавляем запросы к локальному ClickHouse. - if (cluster.getLocalNodesNum() > 0) - { - DB::Context new_context = context; - new_context.setSettings(new_settings); - - const auto & local_addresses = cluster.getLocalShardsInfo(); - for (const auto & address : local_addresses) + if (shard_info.isLocal()) { - InterpreterDescribeQuery interpreter(ast, new_context); - BlockInputStreamPtr stream = new MaterializingBlockInputStream(interpreter.execute().in); - stream = new BlockExtraInfoInputStream(stream, toBlockExtraInfo(address)); + /// Добавляем запросы к локальному ClickHouse. + + DB::Context new_context = context; + new_context.setSettings(new_settings); + + for (const auto & address : shard_info.local_addresses) + { + InterpreterDescribeQuery interpreter(ast, new_context); + BlockInputStreamPtr stream = new MaterializingBlockInputStream(interpreter.execute().in); + stream = new BlockExtraInfoInputStream(stream, toBlockExtraInfo(address)); + res.emplace_back(stream); + } + } + + if (shard_info.hasRemoteConnections()) + { + auto stream = new RemoteBlockInputStream{shard_info.pool, query, &new_settings, throttler}; + stream->reachAllReplicas(); + stream->appendExtraInfo(); res.emplace_back(stream); } } @@ -340,7 +348,7 @@ void StorageDistributed::requireDirectoryMonitor(const std::string & name) size_t StorageDistributed::getShardCount() const { - return cluster.pools.size(); + return cluster.getRemoteShardCount(); } } diff --git a/dbms/src/Storages/System/StorageSystemClusters.cpp b/dbms/src/Storages/System/StorageSystemClusters.cpp index b636e1274ee..43fee761450 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.cpp +++ b/dbms/src/Storages/System/StorageSystemClusters.cpp @@ -72,9 +72,9 @@ BlockInputStreams StorageSystemClusters::read( { const std::string cluster_name = entry.first; const Cluster & cluster = entry.second; - const auto & addresses = cluster.getShardsInfo(); - const auto & addresses_with_failover = cluster.getShardsWithFailoverInfo(); - const auto & shards_info = cluster.shard_info_vec; + const auto & addresses = cluster.getShardsAddresses(); + const auto & addresses_with_failover = cluster.getShardsWithFailoverAddresses(); + const auto & shards_info = cluster.getShardsInfo(); if (!addresses.empty()) { diff --git a/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.reference b/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.reference index d00491fd7e5..6ed281c757a 100644 --- a/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.reference +++ b/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.reference @@ -1 +1,2 @@ 1 +1 diff --git a/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.sql b/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.sql index 822bff89833..fd9e65f9cf3 100644 --- a/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.sql +++ b/dbms/tests/queries/0_stateless/00142_parse_timestamp_as_datetime.sql @@ -1 +1,2 @@ SELECT min(ts = toUInt32(toDateTime(toString(ts)))) FROM (SELECT 1000000000 + 1234 * number AS ts FROM system.numbers LIMIT 1000000); +SELECT min(ts = toUInt32(toDateTime(toString(ts)))) FROM (SELECT 10000 + 1234 * number AS ts FROM system.numbers LIMIT 1000000); diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index f1d2ea9c2a7..d4b61d3da17 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -23,21 +23,186 @@ 1970-01-02 09:00:00 1970-01-02 10:00:00 1970-01-02 18:00:00 1970-01-02 18:00:00 1970-01-02 01:30:00 1970-01-02 01:30:00 +2014 +2014 +2014 +2014 +2014 +9 +9 +9 +10 +9 +30 +30 +30 +1 +30 +2 +2 +2 +3 +2 +23 +21 +20 +4 +11 +50 +50 +50 +50 +50 +0 +0 +0 +0 +0 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014 +2014 +2014 +2014 +2014 +24177 +24177 +24177 +24178 +24177 +2334 +2334 +2334 +2334 +2334 +16343 +16343 +16343 +16344 +16343 +392251 +392251 +392251 +392251 +392251 +23535110 +23535110 +23535110 +23535110 +23535110 +1412106600 +1412106600 +1412106600 +1412106600 +1412106600 +2014-09-30 +2014-09-30 +2014-09-30 +2014-10-01 +2014-09-30 +2014-09-30 +2014-09-30 +2014-09-30 +2014-10-01 +2014-09-30 2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00 2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00 2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00 2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 +2015 7 15 3 +2014 12 28 7 +2015 1 1 4 +2014 9 30 2 +2015 3 15 7 +19 30 0 2015-07-15 13:30:00 +21 0 0 2014-12-29 00:00:00 +12 0 0 2015-01-01 12:00:00 +21 50 0 2014-09-30 23:50:00 +2 30 0 2015-03-15 13:30:00 +2015-07-15 13:00:00 2015 24187 2375 +2014-12-29 00:00:00 2014 24180 2346 +2015-01-01 12:00:00 2015 24181 2347 +2014-09-30 23:00:00 2014 24177 2334 +2015-03-15 13:00:00 2015 24183 2357 +16631 399154 23949270 1436956200 +16432 394389 23663340 1419800400 +16436 394473 23668380 1420102800 +16343 392251 23535110 1412106600 +16509 396226 23773590 1426415400 +2015-07-15 2015-07-15 +2014-12-28 2014-12-28 +2015-01-01 2015-01-01 +2014-09-30 2014-09-30 +2015-03-15 2015-03-15 2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00 2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00 2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00 +2015 7 15 3 +2014 12 29 7 +2015 1 1 4 +2014 9 1 2 +2015 3 15 7 +12 30 0 2015-07-15 13:30:00 +22 0 0 2014-12-29 00:00:00 +10 0 0 2015-01-01 12:00:00 +21 50 0 2014-09-30 23:50:00 +11 30 0 2015-03-15 13:30:00 +2015-07-15 13:00:00 2015 24187 2375 +2014-12-29 00:00:00 2014 24180 2346 +2015-01-01 12:00:00 2015 24181 2347 +2014-09-30 23:00:00 2014 24178 2334 +2015-03-15 13:00:00 2015 24183 2357 +16631 399154 23949270 1436956200 +16432 394389 23663340 1419800400 +16436 394473 23668380 1420102800 +16343 392251 23535110 1412106600 +16509 396226 23773590 1426415400 +2015-07-15 2015-07-15 +2014-12-28 2014-12-28 +2015-01-01 2015-01-01 +2014-09-30 2014-09-30 +2015-03-15 2015-03-15 2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 +2015 3 15 7 +2015 3 15 7 +2015 3 15 7 +2015 3 15 7 +2015 3 15 7 +19 30 0 2015-03-15 13:30:00 +10 30 0 2015-03-15 13:30:00 +13 30 0 2015-03-15 13:30:00 +11 30 0 2015-03-15 13:30:00 +2 30 0 2015-03-15 13:30:00 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 2015-07-15 13:30:00 2015-07-15 12:30:00 2015-07-15 11:30:00 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql index fef2ef11c6b..e694d74befc 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -46,6 +46,148 @@ SELECT toTime(toDateTime(1420102800), 'Europe/London'), toTime(toDateTime(142831 SELECT toTime(toDateTime(1420102800), 'Asia/Tokyo'), toTime(toDateTime(1428310800), 'Asia/Tokyo'); SELECT toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), toTime(toDateTime(1428310800), 'Pacific/Pitcairn'); +/* toYear */ + +SELECT toYear(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toYear(toDateTime(1412106600), 'Europe/Paris'); +SELECT toYear(toDateTime(1412106600), 'Europe/London'); +SELECT toYear(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toYear(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toMonth */ + +SELECT toMonth(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toMonth(toDateTime(1412106600), 'Europe/Paris'); +SELECT toMonth(toDateTime(1412106600), 'Europe/London'); +SELECT toMonth(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toDayOfMonth */ + +SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Paris'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/London'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toDayOfWeek */ + +SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Paris'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/London'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toHour */ + +SELECT toHour(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toHour(toDateTime(1412106600), 'Europe/Paris'); +SELECT toHour(toDateTime(1412106600), 'Europe/London'); +SELECT toHour(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toHour(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toMinute */ + +SELECT toMinute(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toMinute(toDateTime(1412106600), 'Europe/Paris'); +SELECT toMinute(toDateTime(1412106600), 'Europe/London'); +SELECT toMinute(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toMinute(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toSecond */ + +SELECT toSecond(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toSecond(toDateTime(1412106600), 'Europe/Paris'); +SELECT toSecond(toDateTime(1412106600), 'Europe/London'); +SELECT toSecond(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toSecond(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toStartOfMinute */ + +SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/Paris'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/London'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toStartOfHour */ + +SELECT toStartOfHour(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toStartOfHour(toDateTime(1412106600), 'Europe/Paris'); +SELECT toStartOfHour(toDateTime(1412106600), 'Europe/London'); +SELECT toStartOfHour(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toStartOfHour(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeYearNum */ + +SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeMonthNum */ + +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeWeekNum */ + +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeDayNum */ + +SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeHourNum */ + +SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeMinuteNum */ + +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeSecondNum */ + +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toDate */ + +SELECT toDate(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toDate(toDateTime(1412106600), 'Europe/Paris'); +SELECT toDate(toDateTime(1412106600), 'Europe/London'); +SELECT toDate(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toDate(toDateTime(1412106600), 'Pacific/Pitcairn'); + +SELECT toDate(1412106600, 'Europe/Moscow'); +SELECT toDate(1412106600, 'Europe/Paris'); +SELECT toDate(1412106600, 'Europe/London'); +SELECT toDate(1412106600, 'Asia/Tokyo'); +SELECT toDate(1412106600, 'Pacific/Pitcairn'); + DROP TABLE IF EXISTS foo; CREATE TABLE foo(x Int32, y String) ENGINE=Memory; INSERT INTO foo(x, y) VALUES(1420102800, 'Europe/Moscow'); @@ -55,8 +197,25 @@ INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo'); INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn'); SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toYear(toDateTime(x), y), toMonth(toDateTime(x), y), toDayOfMonth(toDateTime(x), y), toDayOfWeek(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toHour(toDateTime(x), y), toMinute(toDateTime(x), y), toSecond(toDateTime(x), y), toStartOfMinute(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toStartOfHour(toDateTime(x), y), toRelativeYearNum(toDateTime(x), y), toRelativeMonthNum(toDateTime(x), y), toRelativeWeekNum(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toRelativeDayNum(toDateTime(x), y), toRelativeHourNum(toDateTime(x), y), toRelativeMinuteNum(toDateTime(x), y), toRelativeSecondNum(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toDate(toDateTime(x), y), toDate(x, y) FROM foo ORDER BY y ASC; + SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC; +SELECT toYear(toDateTime(x), 'Europe/Paris'), toMonth(toDateTime(x), 'Europe/London'), toDayOfMonth(toDateTime(x), 'Asia/Tokyo'), toDayOfWeek(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toHour(toDateTime(x), 'Europe/Paris'), toMinute(toDateTime(x), 'Europe/London'), toSecond(toDateTime(x), 'Asia/Tokyo'), toStartOfMinute(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toStartOfHour(toDateTime(x), 'Europe/Paris'), toRelativeYearNum(toDateTime(x), 'Europe/London'), toRelativeMonthNum(toDateTime(x), 'Asia/Tokyo'), toRelativeWeekNum(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toRelativeDayNum(toDateTime(x), 'Europe/Paris'), toRelativeHourNum(toDateTime(x), 'Europe/London'), toRelativeMinuteNum(toDateTime(x), 'Asia/Tokyo'), toRelativeSecondNum(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toDate(toDateTime(x), 'Europe/Paris'), toDate(x, 'Europe/Paris') FROM foo ORDER BY y ASC; + SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toYear(toDateTime(1426415400), y), toMonth(toDateTime(1426415400), y), toDayOfMonth(toDateTime(1426415400), y), toDayOfWeek(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toHour(toDateTime(1426415400), y), toMinute(toDateTime(1426415400), y), toSecond(toDateTime(1426415400), y), toStartOfMinute(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toStartOfHour(toDateTime(1426415400), y), toRelativeYearNum(toDateTime(1426415400), y), toRelativeMonthNum(toDateTime(1426415400), y), toRelativeWeekNum(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toRelativeDayNum(toDateTime(1426415400), y), toRelativeHourNum(toDateTime(1426415400), y), toRelativeMinuteNum(toDateTime(1426415400), y), toRelativeSecondNum(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toDate(toDateTime(1426415400), y), toDate(1426415400, y) FROM foo ORDER BY y ASC; /* toString */ diff --git a/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference b/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference index e86726625a1..5b1dd0e97cf 100644 --- a/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference +++ b/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference @@ -2,9 +2,9 @@ 1 1 -0 +1 -0 +1 1 1 diff --git a/dbms/tests/queries/0_stateless/00257_no_aggregates_and_constant_keys.reference b/dbms/tests/queries/0_stateless/00257_no_aggregates_and_constant_keys.reference new file mode 100644 index 00000000000..fc77ed8a241 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00257_no_aggregates_and_constant_keys.reference @@ -0,0 +1,29 @@ +40 +41 +2 42 +43 +11 +40 + +40 +41 + +41 +2 42 + +2 42 +43 + +43 +11 + +11 +11 + +11 +1 + +1 +2 + +2 diff --git a/dbms/tests/queries/0_stateless/00257_no_aggregates_and_constant_keys.sql b/dbms/tests/queries/0_stateless/00257_no_aggregates_and_constant_keys.sql new file mode 100644 index 00000000000..15f4536e18f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00257_no_aggregates_and_constant_keys.sql @@ -0,0 +1,15 @@ +select 40 as z from (select * from system.numbers limit 3) group by z; +select 41 as z from remote('127.0.0.{1,2}', system.one) group by z; +select count(), 42 AS z from remote('127.0.0.{1,2}', system.one) group by z; +select 43 AS z from remote('127.0.0.{1,2}', system.one) group by 42, 43, 44; +select 11 AS z from (SELECT 2 UNION ALL SELECT 3) group by 42, 43, 44; + +select 40 as z from (select * from system.numbers limit 3) group by z WITH TOTALS; +select 41 as z from remote('127.0.0.{1,2}', system.one) group by z WITH TOTALS; +select count(), 42 AS z from remote('127.0.0.{1,2}', system.one) group by z WITH TOTALS; +select 43 AS z from remote('127.0.0.{1,2}', system.one) group by 42, 43, 44 WITH TOTALS; +select 11 AS z from (SELECT 1 UNION ALL SELECT 2) group by 42, 43, 44 WITH TOTALS; +select 11 AS z from (SELECT 2 UNION ALL SELECT 3) group by 42, 43, 44 WITH TOTALS; + +SELECT count() WITH TOTALS; +SELECT count() FROM remote('127.0.0.{1,2}', system.one) WITH TOTALS; diff --git a/libs/libcommon/include/ext/map.hpp b/libs/libcommon/include/ext/map.hpp index 6a7941614ec..ac2fbe083f5 100644 --- a/libs/libcommon/include/ext/map.hpp +++ b/libs/libcommon/include/ext/map.hpp @@ -53,7 +53,6 @@ namespace ext * with each element transformed by the application of `mapper`. */ template