diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h index 6a7b30ddbc7..255028c2d05 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -120,6 +120,10 @@ struct AggregateFunctionSequenceMatchData final } }; + +/// Max number of iterations to match the pattern against a sequence, exception thrown when exceeded +constexpr auto sequence_match_max_iterations = 1000000; + class AggregateFunctionSequenceMatch final : public IAggregateFunctionHelper { public: @@ -385,6 +389,7 @@ private: return false; }; + std::size_t i = 0; while (action_it != action_end && events_it != events_end) { // std::cout << "start_timestamp " << base_it->first << "; "; @@ -465,6 +470,12 @@ private: "Unknown PatternActionType", ErrorCodes::LOGICAL_ERROR }; + + if (++i > sequence_match_max_iterations) + throw Exception{ + "Pattern application proves too difficult, exceeding max iterations (" + toString(sequence_match_max_iterations) + ")", + ErrorCodes::TOO_SLOW + }; } /// if there are some actions remaining diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h index 146bb6a9394..3975f238818 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h @@ -25,32 +25,7 @@ namespace DB { - -template struct AggregateFunctionUniqTraits -{ - static UInt64 hash(T x) { return x; } -}; - -template <> struct AggregateFunctionUniqTraits -{ - static UInt64 hash(Float32 x) - { - UInt64 res = 0; - memcpy(reinterpret_cast(&res), reinterpret_cast(&x), sizeof(x)); - return res; - } -}; - -template <> struct AggregateFunctionUniqTraits -{ - static UInt64 hash(Float64 x) - { - UInt64 res = 0; - memcpy(reinterpret_cast(&res), reinterpret_cast(&x), sizeof(x)); - return res; - } -}; - +/// uniq struct AggregateFunctionUniqUniquesHashSetData { @@ -60,6 +35,7 @@ struct AggregateFunctionUniqUniquesHashSetData static String getName() { return "uniq"; } }; +/// uniqHLL12 template struct AggregateFunctionUniqHLL12Data @@ -79,6 +55,7 @@ struct AggregateFunctionUniqHLL12Data static String getName() { return "uniqHLL12"; } }; +/// uniqExact template struct AggregateFunctionUniqExactData @@ -121,8 +98,8 @@ struct AggregateFunctionUniqExactData template struct AggregateFunctionUniqCombinedData { - using Key = T; - using Set = CombinedCardinalityEstimator, HashTableGrower<4> >, 16, 16, 19>; + using Key = UInt32; + using Set = CombinedCardinalityEstimator >, 16, 14, 17, TrivialHash>; Set set; static String getName() { return "uniqCombined"; } @@ -132,7 +109,7 @@ template <> struct AggregateFunctionUniqCombinedData { using Key = UInt64; - using Set = CombinedCardinalityEstimator, HashTableGrower<4> >, 16, 16, 19>; + using Set = CombinedCardinalityEstimator >, 16, 14, 17, TrivialHash>; Set set; static String getName() { return "uniqCombined"; } @@ -140,75 +117,155 @@ struct AggregateFunctionUniqCombinedData namespace detail { - /** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq. - * Используется для частичной специализации для добавления строк. - */ - template - struct OneAdder + +/** Хэш-функция для uniqCombined. + */ +template +struct CombinedCardinalityTraits +{ + static UInt32 hash(T key) { - static void addOne(Data & data, const IColumn & column, size_t row_num) - { - data.set.insert(AggregateFunctionUniqTraits::hash(static_cast &>(column).getData()[row_num])); - } - }; + return key; + } +}; - template - struct OneAdder +template +struct CombinedCardinalityTraits::value>::type> +{ + using U = typename std::make_unsigned::type; + + static UInt32 hash(T key) { - static void addOne(Data & data, const IColumn & column, size_t row_num) - { - /// Имейте ввиду, что вычисление приближённое. - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash64(value.data, value.size)); - } + return intHash32<0>(static_cast(key)); }; +}; - template - struct OneAdder > +template +struct CombinedCardinalityTraits::value>::type> +{ + static UInt32 hash(T key) { - static void addOne(AggregateFunctionUniqExactData & data, const IColumn & column, size_t row_num) - { - data.set.insert(static_cast &>(column).getData()[row_num]); - } + return intHash32<0>(key); }; +}; - template<> - struct OneAdder > +template +struct CombinedCardinalityTraits::value>::type> +{ + static UInt32 hash(T key) { - static void addOne(AggregateFunctionUniqExactData & data, const IColumn & column, size_t row_num) - { - StringRef value = column.getDataAt(row_num); + UInt64 res = 0; + memcpy(reinterpret_cast(&res), reinterpret_cast(&key), sizeof(key)); + return intHash32<0>(res); + } +}; - UInt128 key; - SipHash hash; - hash.update(value.data, value.size); - hash.get128(key.first, key.second); - - data.set.insert(key); - } - }; - - template - struct OneAdder > +template +struct CombinedCardinalityTraits::value>::type> +{ + static UInt32 hash(T key) { - static void addOne(AggregateFunctionUniqCombinedData & data, const IColumn & column, size_t row_num) - { - if (data.set.isMedium()) - data.set.insert(static_cast &>(column).getData()[row_num]); - else - data.set.insert(AggregateFunctionUniqTraits::hash(static_cast &>(column).getData()[row_num])); - } - }; + UInt32 res = 0; + memcpy(reinterpret_cast(&res), reinterpret_cast(&key), sizeof(key)); + return res; + } +}; - template<> - struct OneAdder > +/** Хэш-функция для uniq. + */ +template struct AggregateFunctionUniqTraits +{ + static UInt64 hash(T x) { return x; } +}; + +template <> struct AggregateFunctionUniqTraits +{ + static UInt64 hash(Float32 x) { - static void addOne(AggregateFunctionUniqCombinedData & data, const IColumn & column, size_t row_num) - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash64(value.data, value.size)); - } - }; + UInt64 res = 0; + memcpy(reinterpret_cast(&res), reinterpret_cast(&x), sizeof(x)); + return res; + } +}; + +template <> struct AggregateFunctionUniqTraits +{ + static UInt64 hash(Float64 x) + { + UInt64 res = 0; + memcpy(reinterpret_cast(&res), reinterpret_cast(&x), sizeof(x)); + return res; + } +}; + +/** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq. + * Используется для частичной специализации для добавления строк. + */ +template +struct OneAdder +{ + static void addOne(Data & data, const IColumn & column, size_t row_num) + { + data.set.insert(AggregateFunctionUniqTraits::hash(static_cast &>(column).getData()[row_num])); + } +}; + +template +struct OneAdder +{ + static void addOne(Data & data, const IColumn & column, size_t row_num) + { + /// Имейте ввиду, что вычисление приближённое. + StringRef value = column.getDataAt(row_num); + data.set.insert(CityHash64(value.data, value.size)); + } +}; + +template +struct OneAdder > +{ + static void addOne(AggregateFunctionUniqExactData & data, const IColumn & column, size_t row_num) + { + data.set.insert(static_cast &>(column).getData()[row_num]); + } +}; + +template<> +struct OneAdder > +{ + static void addOne(AggregateFunctionUniqExactData & data, const IColumn & column, size_t row_num) + { + StringRef value = column.getDataAt(row_num); + + UInt128 key; + SipHash hash; + hash.update(value.data, value.size); + hash.get128(key.first, key.second); + + data.set.insert(key); + } +}; + +template +struct OneAdder > +{ + static void addOne(AggregateFunctionUniqCombinedData & data, const IColumn & column, size_t row_num) + { + const auto & value = static_cast &>(column).getData()[row_num]; + data.set.insert(CombinedCardinalityTraits::hash(value)); + } +}; + +template<> +struct OneAdder > +{ + static void addOne(AggregateFunctionUniqCombinedData & data, const IColumn & column, size_t row_num) + { + StringRef value = column.getDataAt(row_num); + data.set.insert(CityHash64(value.data, value.size)); + } +}; + } diff --git a/dbms/include/DB/Common/Allocator.h b/dbms/include/DB/Common/Allocator.h new file mode 100644 index 00000000000..524d88f010f --- /dev/null +++ b/dbms/include/DB/Common/Allocator.h @@ -0,0 +1,124 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include + + +/** Отвечает за выделение/освобождение памяти. Используется, например, в PODArray, Arena. + * Интерфейс отличается от std::allocator + * - наличием метода realloc, который для больших кусков памяти использует mremap; + * - передачей размера в метод free; + * - наличием аргумента alignment; + */ +class Allocator +{ +private: + /** См. комментарий в HashTableAllocator.h + */ + static constexpr size_t MMAP_THRESHOLD = 64 * (1 << 20); + static constexpr size_t HUGE_PAGE_SIZE = 2 * (1 << 20); + static constexpr size_t MMAP_MIN_ALIGNMENT = 4096; + static constexpr size_t MALLOC_MIN_ALIGNMENT = 8; + +public: + /// Выделить кусок памяти. + void * alloc(size_t size, size_t alignment = 0) + { + if (current_memory_tracker) + current_memory_tracker->alloc(size); + + void * buf; + + if (size >= MMAP_THRESHOLD) + { + if (alignment > MMAP_MIN_ALIGNMENT) + throw DB::Exception("Too large alignment: more than page size.", DB::ErrorCodes::BAD_ARGUMENTS); + + buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + /// См. комментарий в HashTableAllocator.h + if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE)) + DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } + else + { + if (alignment <= MALLOC_MIN_ALIGNMENT) + { + buf = ::malloc(size); + + if (nullptr == buf) + DB::throwFromErrno("Allocator: Cannot malloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } + else + { + buf = nullptr; + int res = posix_memalign(&buf, alignment, size); + + if (0 != res) + DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + } + } + + return buf; + } + + /// Освободить память. + void free(void * buf, size_t size) + { + if (size >= MMAP_THRESHOLD) + { + if (0 != munmap(buf, size)) + DB::throwFromErrno("Allocator: Cannot munmap.", DB::ErrorCodes::CANNOT_MUNMAP); + } + else + { + ::free(buf); + } + + if (current_memory_tracker) + current_memory_tracker->free(size); + } + + /** Увеличить размер куска памяти. + * Содержимое старого куска памяти переезжает в начало нового. + * Положение куска памяти может измениться. + */ + void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0) + { + if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD && alignment <= MALLOC_MIN_ALIGNMENT) + { + if (current_memory_tracker) + current_memory_tracker->realloc(old_size, new_size); + + buf = ::realloc(buf, new_size); + + if (nullptr == buf) + DB::throwFromErrno("Allocator: Cannot realloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } + else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD) + { + if (current_memory_tracker) + current_memory_tracker->realloc(old_size, new_size); + + buf = mremap(buf, old_size, new_size, MREMAP_MAYMOVE); + if (MAP_FAILED == buf) + DB::throwFromErrno("Allocator: Cannot mremap.", DB::ErrorCodes::CANNOT_MREMAP); + } + else + { + void * new_buf = alloc(new_size, alignment); + memcpy(new_buf, buf, old_size); + free(buf, old_size); + buf = new_buf; + } + + return buf; + } +}; diff --git a/dbms/include/DB/Common/Arena.h b/dbms/include/DB/Common/Arena.h index 5e44bf7df70..a1b0211452e 100644 --- a/dbms/include/DB/Common/Arena.h +++ b/dbms/include/DB/Common/Arena.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB @@ -25,7 +25,7 @@ class Arena { private: /// Непрерывный кусок памяти и указатель на свободное место в нём. Односвязный список. - struct Chunk : private std::allocator /// empty base optimization + struct Chunk : private Allocator /// empty base optimization { char * begin; char * pos; @@ -38,10 +38,7 @@ private: ProfileEvents::increment(ProfileEvents::ArenaAllocChunks); ProfileEvents::increment(ProfileEvents::ArenaAllocBytes, size_); - if (current_memory_tracker) - current_memory_tracker->alloc(size_); - - begin = allocate(size_); + begin = reinterpret_cast(Allocator::alloc(size_)); pos = begin; end = begin + size_; prev = prev_; @@ -49,10 +46,7 @@ private: ~Chunk() { - deallocate(begin, size()); - - if (current_memory_tracker) - current_memory_tracker->free(size()); + Allocator::free(begin, size()); if (prev) delete prev; diff --git a/dbms/include/DB/Common/CombinedCardinalityEstimator.h b/dbms/include/DB/Common/CombinedCardinalityEstimator.h index 43c11380668..c6a2ff2d119 100644 --- a/dbms/include/DB/Common/CombinedCardinalityEstimator.h +++ b/dbms/include/DB/Common/CombinedCardinalityEstimator.h @@ -2,7 +2,8 @@ #include #include -#include +#include +#include namespace DB @@ -11,11 +12,11 @@ namespace DB namespace details { -enum class ContainerType { SMALL, MEDIUM, LARGE }; +enum class ContainerType : UInt8 { SMALL = 1, MEDIUM = 2, LARGE = 3 }; -ContainerType max(const ContainerType & lhs, const ContainerType & rhs) +static inline ContainerType max(const ContainerType & lhs, const ContainerType & rhs) { - unsigned int res = std::max(static_cast(lhs), static_cast(rhs)); + UInt8 res = std::max(static_cast(lhs), static_cast(rhs)); return static_cast(res); } @@ -25,38 +26,41 @@ ContainerType max(const ContainerType & lhs, const ContainerType & rhs) * Для среднего - выделяется HashSet. * Для большого - выделяется HyperLogLog. */ -template +template +< + typename Key, + typename HashContainer, + UInt8 small_set_size_max, + UInt8 medium_set_power2_max, + UInt8 K, + typename Hash = IntHash32, + typename DenominatorType = double +> class CombinedCardinalityEstimator { public: - using Self = CombinedCardinalityEstimator; + using Self = CombinedCardinalityEstimator; private: using Small = SmallSet; using Medium = HashContainer; - using Large = HyperLogLogWithSmallSetOptimization; + using Large = HyperLogLogCounter; public: + CombinedCardinalityEstimator() + { + setContainerType(details::ContainerType::SMALL); + } + ~CombinedCardinalityEstimator() { - if (container_type == details::ContainerType::MEDIUM) - { - delete medium; - - if (current_memory_tracker) - current_memory_tracker->free(sizeof(medium)); - } - else if (container_type == details::ContainerType::LARGE) - { - delete large; - - if (current_memory_tracker) - current_memory_tracker->free(sizeof(large)); - } + destroy(); } void insert(Key value) { + auto container_type = getContainerType(); + if (container_type == details::ContainerType::SMALL) { if (small.find(value) == small.end()) @@ -66,41 +70,43 @@ public: else { toMedium(); - medium->insert(value); + getContainer().insert(value); } } } else if (container_type == details::ContainerType::MEDIUM) { - if (medium->size() < medium_set_size_max) - medium->insert(value); + auto & container = getContainer(); + if (container.size() < medium_set_size_max) + container.insert(value); else { toLarge(); - large->insert(value); + getContainer().insert(value); } } else if (container_type == details::ContainerType::LARGE) - large->insert(value); - else - throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); + getContainer().insert(value); } UInt32 size() const { + auto container_type = getContainerType(); + if (container_type == details::ContainerType::SMALL) return small.size(); else if (container_type == details::ContainerType::MEDIUM) - return medium->size(); + return getContainer().size(); else if (container_type == details::ContainerType::LARGE) - return large->size(); + return getContainer().size(); else throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); } void merge(const Self & rhs) { - details::ContainerType max_container_type = details::max(container_type, rhs.container_type); + auto container_type = getContainerType(); + auto max_container_type = details::max(container_type, rhs.getContainerType()); if (container_type != max_container_type) { @@ -110,41 +116,18 @@ public: toLarge(); } - if (container_type == details::ContainerType::SMALL) + if (rhs.getContainerType() == details::ContainerType::SMALL) { for (const auto & x : rhs.small) insert(x); } - else if (container_type == details::ContainerType::MEDIUM) + else if (rhs.getContainerType() == details::ContainerType::MEDIUM) { - if (rhs.container_type == details::ContainerType::SMALL) - { - for (const auto & x : rhs.small) - insert(x); - } - else if (rhs.container_type == details::ContainerType::MEDIUM) - { - for (const auto & x : *rhs.medium) - insert(x); - } + for (const auto & x : rhs.getContainer()) + insert(x); } - else if (container_type == details::ContainerType::LARGE) - { - if (rhs.container_type == details::ContainerType::SMALL) - { - for (const auto & x : rhs.small) - insert(x); - } - else if (rhs.container_type == details::ContainerType::MEDIUM) - { - for (const auto & x : *rhs.medium) - insert(x); - } - else if (rhs.container_type == details::ContainerType::LARGE) - large->merge(*rhs.large); - } - else - throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); + else if (rhs.getContainerType() == details::ContainerType::LARGE) + getContainer().merge(rhs.getContainer()); } /// Можно вызывать только для пустого объекта. @@ -152,79 +135,95 @@ public: { UInt8 v; readBinary(v, in); - details::ContainerType t = static_cast(v); + auto container_type = static_cast(v); - if (t == details::ContainerType::SMALL) + if (container_type == details::ContainerType::SMALL) small.read(in); - else if (t == details::ContainerType::MEDIUM) + else if (container_type == details::ContainerType::MEDIUM) { toMedium(); - medium->read(in); + getContainer().read(in); } - else if (t == details::ContainerType::LARGE) + else if (container_type == details::ContainerType::LARGE) { toLarge(); - large->read(in); + getContainer().read(in); } - else - throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); } void readAndMerge(DB::ReadBuffer & in) { - Self other; - other.read(in); - merge(other); + auto container_type = getContainerType(); + + UInt8 v; + readBinary(v, in); + auto rhs_container_type = static_cast(v); + + auto max_container_type = details::max(container_type, rhs_container_type); + + if (container_type != max_container_type) + { + if (max_container_type == details::ContainerType::MEDIUM) + toMedium(); + else if (max_container_type == details::ContainerType::LARGE) + toLarge(); + } + + if (rhs_container_type == details::ContainerType::SMALL) + { + typename Small::Reader reader(in); + while (reader.next()) + insert(reader.get()); + } + else if (rhs_container_type == details::ContainerType::MEDIUM) + { + typename Medium::Reader reader(in); + while (reader.next()) + insert(reader.get()); + } + else if (rhs_container_type == details::ContainerType::LARGE) + getContainer().readAndMerge(in); } void write(DB::WriteBuffer & out) const { - UInt8 v = static_cast(container_type); - writeBinary(v, out); + auto container_type = getContainerType(); + writeBinary(static_cast(container_type), out); if (container_type == details::ContainerType::SMALL) small.write(out); else if (container_type == details::ContainerType::MEDIUM) - medium->write(out); + getContainer().write(out); else if (container_type == details::ContainerType::LARGE) - large->write(out); - else - throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); - } - - bool isMedium() const - { - return container_type == details::ContainerType::MEDIUM; + getContainer().write(out); } private: void toMedium() { - if (container_type != details::ContainerType::SMALL) + if (getContainerType() != details::ContainerType::SMALL) throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); - if (current_memory_tracker) - current_memory_tracker->alloc(sizeof(medium)); - - Medium * tmp_medium = new Medium; + auto tmp_medium = std::make_unique(); for (const auto & x : small) tmp_medium->insert(x); - medium = tmp_medium; + medium = tmp_medium.release(); + setContainerType(details::ContainerType::MEDIUM); - container_type = details::ContainerType::MEDIUM; + if (current_memory_tracker) + current_memory_tracker->alloc(sizeof(medium)); } void toLarge() { + auto container_type = getContainerType(); + if ((container_type != details::ContainerType::SMALL) && (container_type != details::ContainerType::MEDIUM)) throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR); - if (current_memory_tracker) - current_memory_tracker->alloc(sizeof(large)); - - Large * tmp_large = new Large; + auto tmp_large = std::make_unique(); if (container_type == details::ContainerType::SMALL) { @@ -233,11 +232,25 @@ private: } else if (container_type == details::ContainerType::MEDIUM) { - for (const auto & x : *medium) + for (const auto & x : getContainer()) tmp_large->insert(x); + + destroy(); } - large = tmp_large; + large = tmp_large.release(); + setContainerType(details::ContainerType::LARGE); + + if (current_memory_tracker) + current_memory_tracker->alloc(sizeof(large)); + + } + + void NO_INLINE destroy() + { + auto container_type = getContainerType(); + + clearContainerType(); if (container_type == details::ContainerType::MEDIUM) { @@ -247,16 +260,54 @@ private: if (current_memory_tracker) current_memory_tracker->free(sizeof(medium)); } + else if (container_type == details::ContainerType::LARGE) + { + delete large; + large = nullptr; - container_type = details::ContainerType::LARGE; + if (current_memory_tracker) + current_memory_tracker->free(sizeof(large)); + } + } + + template + inline T & getContainer() + { + return *reinterpret_cast(address & mask); + } + + template + inline const T & getContainer() const + { + return *reinterpret_cast(address & mask); + } + + void setContainerType(details::ContainerType t) + { + address &= mask; + address |= static_cast(t); + } + + inline details::ContainerType getContainerType() const + { + return static_cast(address & ~mask); + } + + void clearContainerType() + { + address &= mask; } private: Small small; - Medium * medium = nullptr; - Large * large = nullptr; - const UInt32 medium_set_size_max = 1UL << medium_set_power2_max; - details::ContainerType container_type = details::ContainerType::SMALL; + union + { + Medium * medium; + Large * large; + UInt64 address = 0; + }; + static const UInt64 mask = 0xFFFFFFFFFFFFFFFC; + static const UInt32 medium_set_size_max = 1UL << medium_set_power2_max; }; } diff --git a/dbms/include/DB/Common/FileChecker.h b/dbms/include/DB/Common/FileChecker.h index 88e2fa18eab..4e37ef0b16f 100644 --- a/dbms/include/DB/Common/FileChecker.h +++ b/dbms/include/DB/Common/FileChecker.h @@ -16,12 +16,11 @@ namespace DB { /// хранит размеры всех столбцов, и может проверять не побились ли столбцы -template class FileChecker { public: - FileChecker(const std::string &file_info_path_, Storage & storage_) : - files_info_path(file_info_path_), files_info(), storage(storage_), log(&Logger::get("FileChecker")) + FileChecker(const std::string & file_info_path_) : + files_info_path(file_info_path_), files_info(), log(&Logger::get("FileChecker")) { Poco::Path path(files_info_path); tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName(); @@ -107,7 +106,6 @@ private: using PropertyTree = boost::property_tree::ptree; PropertyTree files_info; - Storage & storage; Logger * log; }; } diff --git a/dbms/include/DB/Common/HashTable/HashTable.h b/dbms/include/DB/Common/HashTable/HashTable.h index 67196746ae5..02c157035a8 100644 --- a/dbms/include/DB/Common/HashTable/HashTable.h +++ b/dbms/include/DB/Common/HashTable/HashTable.h @@ -251,6 +251,7 @@ class HashTable : protected: friend class const_iterator; friend class iterator; + friend class Reader; template friend class TwoLevelHashTable; @@ -429,6 +430,51 @@ public: free(); } + class Reader final : private Cell::State + { + public: + Reader(DB::ReadBuffer & in_) + : in(in_) + { + } + + Reader(const Reader &) = delete; + Reader & operator=(const Reader &) = delete; + + bool next() + { + if (read_count == size) + { + is_eof = true; + return false; + } + else if (read_count == 0) + { + Cell::State::read(in); + DB::readVarUInt(size, in); + } + + cell.read(in); + ++read_count; + + return true; + } + + inline const value_type & get() const + { + if ((read_count == 0) || is_eof) + throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA); + + return cell.getValue(); + } + + private: + DB::ReadBuffer in; + Cell cell; + size_t read_count = 0; + size_t size; + bool is_eof = false; + }; class iterator { @@ -757,7 +803,7 @@ public: { Cell x; x.read(rb); - insert(Cell::getKey(x.getValue())); + insert(x); } } @@ -781,7 +827,7 @@ public: Cell x; DB::assertString(",", rb); x.readText(rb); - insert(Cell::getKey(x.getValue())); + insert(x); } } diff --git a/dbms/include/DB/Common/HashTable/HashTableAllocator.h b/dbms/include/DB/Common/HashTable/HashTableAllocator.h index e3d9b462c39..5c36857406c 100644 --- a/dbms/include/DB/Common/HashTable/HashTableAllocator.h +++ b/dbms/include/DB/Common/HashTable/HashTableAllocator.h @@ -19,6 +19,7 @@ /** Общая часть разных хэш-таблиц, отвечающая за выделение/освобождение памяти. + * Отличается от Allocator тем, что зануляет память. * Используется в качестве параметра шаблона (есть несколько реализаций с таким же интерфейсом). */ class HashTableAllocator @@ -33,9 +34,9 @@ private: * Рассчитываем, что набор операций mmap/что-то сделать/mremap может выполняться всего лишь около 1000 раз в секунду. * * PS. Также это требуется, потому что tcmalloc не может выделить кусок памяти больше 16 GB. - * NOTE Можно попробовать MAP_HUGETLB, но придётся самостоятельно управлять количеством доступных страниц. */ static constexpr size_t MMAP_THRESHOLD = 64 * (1 << 20); + static constexpr size_t HUGE_PAGE_SIZE = 2 * (1 << 20); public: /// Выделить кусок памяти и заполнить его нулями. @@ -52,6 +53,14 @@ public: if (MAP_FAILED == buf) DB::throwFromErrno("HashTableAllocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + /** Использование huge pages позволяет увеличить производительность более чем в три раза + * в запросе SELECT number % 1000000 AS k, count() FROM system.numbers GROUP BY k, + * (хэш-таблица на 1 000 000 элементов) + * и примерно на 15% в случае хэш-таблицы на 100 000 000 элементов. + */ + if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE)) + DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + /// Заполнение нулями не нужно - mmap сам это делает. } else @@ -108,6 +117,10 @@ public: if (MAP_FAILED == buf) DB::throwFromErrno("HashTableAllocator: Cannot mremap.", DB::ErrorCodes::CANNOT_MREMAP); + /** Здесь не получается сделать madvise с MADV_HUGEPAGE. + * Похоже, что при mremap, huge pages сами расширяются на новую область. + */ + /// Заполнение нулями не нужно. } else diff --git a/dbms/include/DB/Common/HashTable/SmallTable.h b/dbms/include/DB/Common/HashTable/SmallTable.h index 10ec8479b93..c68963a4798 100644 --- a/dbms/include/DB/Common/HashTable/SmallTable.h +++ b/dbms/include/DB/Common/HashTable/SmallTable.h @@ -27,6 +27,7 @@ class SmallTable : protected: friend class const_iterator; friend class iterator; + friend class Reader; typedef SmallTable Self; typedef Cell cell_type; @@ -66,6 +67,55 @@ public: typedef typename Cell::value_type value_type; + class Reader final : private Cell::State + { + public: + Reader(DB::ReadBuffer & in_) + : in(in_) + { + } + + Reader(const Reader &) = delete; + Reader & operator=(const Reader &) = delete; + + bool next() + { + if (read_count == size) + { + is_eof = true; + return false; + } + else if (read_count == 0) + { + Cell::State::read(in); + DB::readVarUInt(size, in); + + if (size > capacity) + throw DB::Exception("Illegal size"); + } + + cell.read(in); + ++read_count; + + return true; + } + + inline const value_type & get() const + { + if ((read_count == 0) || is_eof) + throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA); + + return cell.getValue(); + } + + private: + DB::ReadBuffer in; + Cell cell; + size_t read_count = 0; + size_t size; + bool is_eof = false; + }; + class iterator { Self * container; diff --git a/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h b/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h index 7932ddfb0e8..e95811ce27b 100644 --- a/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h +++ b/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h @@ -16,7 +16,7 @@ template < UInt8 small_set_size, UInt8 K, typename Hash = IntHash32, - typename DenominatorType = float> + typename DenominatorType = double> class HyperLogLogWithSmallSetOptimization { private: @@ -114,10 +114,20 @@ public: void readAndMerge(DB::ReadBuffer & in) { - /// Немного не оптимально. - HyperLogLogWithSmallSetOptimization other; - other.read(in); - merge(other); + bool is_rhs_large; + readBinary(is_rhs_large, in); + + if (!isLarge() && is_rhs_large) + toLarge(); + + if (!is_rhs_large) + { + typename Small::Reader reader(in); + while (reader.next()) + insert(reader.get()); + } + else + large->readAndMerge(in); } void write(DB::WriteBuffer & out) const diff --git a/dbms/include/DB/Common/PODArray.h b/dbms/include/DB/Common/PODArray.h index 7d96889483e..7f0ef5758da 100644 --- a/dbms/include/DB/Common/PODArray.h +++ b/dbms/include/DB/Common/PODArray.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -12,7 +11,7 @@ #include #include -#include +#include #include #include @@ -32,28 +31,18 @@ namespace DB * Конструктор по-умолчанию создаёт пустой объект, который не выделяет память. * Затем выделяется память минимум под POD_ARRAY_INITIAL_SIZE элементов. * - * При первом выделении памяти использует std::allocator. - * В реализации из libstdc++ он кэширует куски памяти несколько больше, чем обычный malloc. - * - * При изменении размера, использует realloc, который может (но не обязан) использовать mremap для больших кусков памяти. - * По факту, mremap используется при использовании аллокатора из glibc, но не используется, например, в tcmalloc. - * * Если вставлять элементы push_back-ом, не делая reserve, то PODArray примерно в 2.5 раза быстрее std::vector. */ #define POD_ARRAY_INITIAL_SIZE 4096UL template -class PODArray : private boost::noncopyable, private std::allocator /// empty base optimization +class PODArray : private boost::noncopyable, private Allocator /// empty base optimization { private: - typedef std::allocator Allocator; - char * c_start; char * c_end; char * c_end_of_storage; - bool use_libc_realloc = false; - T * t_start() { return reinterpret_cast(c_start); } T * t_end() { return reinterpret_cast(c_end); } T * t_end_of_storage() { return reinterpret_cast(c_end_of_storage); } @@ -90,10 +79,7 @@ private: size_t bytes_to_alloc = to_size(n); - if (current_memory_tracker) - current_memory_tracker->alloc(bytes_to_alloc); - - c_start = c_end = Allocator::allocate(bytes_to_alloc); + c_start = c_end = reinterpret_cast(Allocator::alloc(bytes_to_alloc)); c_end_of_storage = c_start + bytes_to_alloc; } @@ -102,13 +88,7 @@ private: if (c_start == nullptr) return; - if (use_libc_realloc) - ::free(c_start); - else - Allocator::deallocate(c_start, storage_size()); - - if (current_memory_tracker) - current_memory_tracker->free(storage_size()); + Allocator::free(c_start, storage_size()); } void realloc(size_t n) @@ -122,38 +102,10 @@ private: ptrdiff_t end_diff = c_end - c_start; size_t bytes_to_alloc = to_size(n); - char * old_c_start = c_start; - char * old_c_end_of_storage = c_end_of_storage; - - if (current_memory_tracker) - current_memory_tracker->realloc(storage_size(), bytes_to_alloc); - - if (use_libc_realloc) - { - auto new_c_start = reinterpret_cast(::realloc(c_start, bytes_to_alloc)); - - if (nullptr == new_c_start) - throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY); - - c_start = new_c_start; - } - else - { - auto new_c_start = reinterpret_cast(malloc(bytes_to_alloc)); - - if (nullptr == new_c_start) - throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY); - - c_start = new_c_start; - - memcpy(c_start, old_c_start, std::min(bytes_to_alloc, static_cast(end_diff))); - Allocator::deallocate(old_c_start, old_c_end_of_storage - old_c_start); - } + c_start = reinterpret_cast(Allocator::realloc(c_start, storage_size(), bytes_to_alloc)); c_end = c_start + end_diff; c_end_of_storage = c_start + bytes_to_alloc; - - use_libc_realloc = true; } public: @@ -187,7 +139,6 @@ public: std::swap(c_start, other.c_start); std::swap(c_end, other.c_end); std::swap(c_end_of_storage, other.c_end_of_storage); - std::swap(use_libc_realloc, other.use_libc_realloc); return *this; } diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index 937b06d5ce0..c10f878ab4d 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -283,6 +283,9 @@ namespace ErrorCodes INDEX_NOT_USED = 277, LEADERSHIP_LOST = 278, ALL_CONNECTION_TRIES_FAILED = 279, + NO_AVAILABLE_DATA = 280, + DICTIONARY_IS_EMPTY = 281, + INCORRECT_INDEX = 282, KEEPER_EXCEPTION = 999, POCO_EXCEPTION = 1000, diff --git a/dbms/include/DB/Core/Exception.h b/dbms/include/DB/Core/Exception.h index 3d28e412137..e9d8ec3c737 100644 --- a/dbms/include/DB/Core/Exception.h +++ b/dbms/include/DB/Core/Exception.h @@ -28,8 +28,8 @@ ExceptionPtr cloneCurrentException(); /** Попробовать записать исключение в лог (и забыть про него). * Можно использовать в деструкторах в блоке catch (...). */ -void tryLogCurrentException(const char * log_name); -void tryLogCurrentException(Poco::Logger * logger); +void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); +void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); std::string getCurrentExceptionMessage(bool with_stacktrace); diff --git a/dbms/include/DB/DataStreams/FormatFactory.h b/dbms/include/DB/DataStreams/FormatFactory.h index d8e224e1cb0..848f84828af 100644 --- a/dbms/include/DB/DataStreams/FormatFactory.h +++ b/dbms/include/DB/DataStreams/FormatFactory.h @@ -14,10 +14,10 @@ class FormatFactory { public: BlockInputStreamPtr getInput(const String & name, ReadBuffer & buf, - Block & sample, size_t max_block_size) const; + const Block & sample, size_t max_block_size) const; BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, - Block & sample) const; + const Block & sample) const; }; } diff --git a/dbms/include/DB/DataStreams/MarkInCompressedFile.h b/dbms/include/DB/DataStreams/MarkInCompressedFile.h new file mode 100644 index 00000000000..ff21cbb8af4 --- /dev/null +++ b/dbms/include/DB/DataStreams/MarkInCompressedFile.h @@ -0,0 +1,38 @@ +#pragma once + +#include + +#include +#include + + +namespace DB +{ + +/** Засечка - позиция в сжатом файле. Сжатый файл состоит из уложенных подряд сжатых блоков. + * Засечка представляют собой пару - смещение в файле до начала сжатого блока, смещение в разжатом блоке до начала данных. + */ +struct MarkInCompressedFile +{ + size_t offset_in_compressed_file; + size_t offset_in_decompressed_block; + + bool operator==(const MarkInCompressedFile & rhs) const + { + return std::tie(offset_in_compressed_file, offset_in_decompressed_block) + == std::tie(rhs.offset_in_compressed_file, rhs.offset_in_decompressed_block); + } + bool operator!=(const MarkInCompressedFile & rhs) const + { + return !(*this == rhs); + } + + String toString() const + { + return "(" + DB::toString(offset_in_compressed_file) + "," + DB::toString(offset_in_decompressed_block) + ")"; + } +}; + +using MarksInCompressedFile = std::vector; + +} diff --git a/dbms/include/DB/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/include/DB/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h new file mode 100644 index 00000000000..cbe117dbcdb --- /dev/null +++ b/dbms/include/DB/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -0,0 +1,110 @@ +#pragma once + +#include +#include + + +namespace DB +{ + + +/** Доагрегирует потоки блоков, держа в оперативной памяти только по одному блоку из каждого потока. + * Это экономит оперативку в случае использования двухуровневой агрегации, где в каждом потоке будет до 256 блоков с частями результата. + * + * Агрегатные функции в блоках не должны быть финализированы, чтобы их состояния можно было объединить. + */ +class MergingAggregatedMemoryEfficientBlockInputStream : public IProfilingBlockInputStream +{ +public: + MergingAggregatedMemoryEfficientBlockInputStream(BlockInputStreams inputs_, const Names & keys_names_, + const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_) + : aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0), + final(final_) + { + children = inputs_; + current_blocks.resize(children.size()); + overflow_blocks.resize(children.size()); + is_exhausted.resize(children.size()); + } + + String getName() const override { return "MergingAggregatedMemorySavvy"; } + + String getID() const override + { + std::stringstream res; + res << "MergingAggregatedMemorySavvy(" << aggregator.getID(); + for (size_t i = 0, size = children.size(); i < size; ++i) + res << ", " << children.back()->getID(); + res << ")"; + return res.str(); + } + +protected: + Block readImpl() override + { + /// Если child - RemoteBlockInputStream, то отправляет запрос на все удалённые серверы, инициируя вычисления. + if (current_bucket_num == -1) + for (auto & child : children) + child->readPrefix(); + + /// Всё прочитали. + if (current_bucket_num > 255) + return {}; + + /// Читаем следующие блоки для current_bucket_num + for (size_t i = 0, size = children.size(); i < size; ++i) + { + while (!is_exhausted[i] && (!current_blocks[i] || current_blocks[i].info.bucket_num < current_bucket_num)) + { + current_blocks[i] = children[i]->read(); + + if (!current_blocks[i]) + { + is_exhausted[i] = true; + } + else if (current_blocks[i].info.is_overflows) + { + overflow_blocks[i].swap(current_blocks[i]); + } + } + } + + /// Может быть, нет блоков для current_bucket_num, а все блоки имеют больший bucket_num. + Int32 min_bucket_num = 256; + for (size_t i = 0, size = children.size(); i < size; ++i) + if (!is_exhausted[i] && current_blocks[i].info.bucket_num < min_bucket_num) + min_bucket_num = current_blocks[i].info.bucket_num; + + current_bucket_num = min_bucket_num; + + /// Все потоки исчерпаны. + if (current_bucket_num > 255) + return {}; /// TODO overflow_blocks. + + /// TODO Если есть single_level и two_level блоки. + + /// Объединяем все блоки с current_bucket_num. + + BlocksList blocks_to_merge; + for (size_t i = 0, size = children.size(); i < size; ++i) + if (current_blocks[i].info.bucket_num == current_bucket_num) + blocks_to_merge.emplace_back(std::move(current_blocks[i])); + + Block res = aggregator.mergeBlocks(blocks_to_merge, final); + + ++current_bucket_num; + return res; + } + +private: + Aggregator aggregator; + bool final; + + Int32 current_bucket_num = -1; + std::vector current_blocks; + std::vector is_exhausted; + + std::vector overflow_blocks; +}; + +} diff --git a/dbms/include/DB/DataStreams/NativeBlockInputStream.h b/dbms/include/DB/DataStreams/NativeBlockInputStream.h index c9810b55385..f69f0b91329 100644 --- a/dbms/include/DB/DataStreams/NativeBlockInputStream.h +++ b/dbms/include/DB/DataStreams/NativeBlockInputStream.h @@ -1,22 +1,75 @@ #pragma once #include +#include namespace DB { +class CompressedReadBufferFromFile; + + +/** Формат Native может содержать отдельно расположенный индекс, + * который позволяет понять, где какой столбец расположен, + * и пропускать ненужные столбцы. + */ + +/** Позиция одного кусочка одного столбца. */ +struct IndexOfOneColumnForNativeFormat +{ + String name; + String type; + MarkInCompressedFile location; +}; + +/** Индекс для блока данных. */ +struct IndexOfBlockForNativeFormat +{ + using Columns = std::vector; + + size_t num_columns; + size_t num_rows; + Columns columns; +}; + +/** Весь индекс. */ +struct IndexForNativeFormat +{ + using Blocks = std::vector; + Blocks blocks; + + IndexForNativeFormat() {} + + IndexForNativeFormat(ReadBuffer & istr, const NameSet & required_columns) + { + read(istr, required_columns); + } + + /// Прочитать индекс, только для нужных столбцов. + void read(ReadBuffer & istr, const NameSet & required_columns); +}; + + /** Десериализует поток блоков из родного бинарного формата (с именами и типами столбцов). * Предназначено для взаимодействия между серверами. + * + * Также может использоваться для хранения данных на диске. + * В этом случае, может использовать индекс. */ class NativeBlockInputStream : public IProfilingBlockInputStream { public: /** В случае указания ненулевой server_revision, может ожидаться и считываться дополнительная информация о блоке, * в зависимости от поддерживаемой для указанной ревизии. + * + * index - не обязательный параметр. Если задан, то будут читаться только указанные в индексе кусочки столбцов. */ - NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_ = 0) - : istr(istr_), server_revision(server_revision_) {} + NativeBlockInputStream( + ReadBuffer & istr_, UInt64 server_revision_ = 0, + bool use_index_ = false, + IndexForNativeFormat::Blocks::const_iterator index_block_it_ = IndexForNativeFormat::Blocks::const_iterator{}, + IndexForNativeFormat::Blocks::const_iterator index_block_end_ = IndexForNativeFormat::Blocks::const_iterator{}); String getName() const override { return "Native"; } @@ -35,6 +88,14 @@ protected: private: ReadBuffer & istr; UInt64 server_revision; + + bool use_index; + IndexForNativeFormat::Blocks::const_iterator index_block_it; + IndexForNativeFormat::Blocks::const_iterator index_block_end; + IndexOfBlockForNativeFormat::Columns::const_iterator index_column_it; + + /// Если задан индекс, то istr должен быть CompressedReadBufferFromFile. + CompressedReadBufferFromFile * istr_concrete; }; } diff --git a/dbms/include/DB/DataStreams/NativeBlockOutputStream.h b/dbms/include/DB/DataStreams/NativeBlockOutputStream.h index e0d2c3023f8..b39444f06be 100644 --- a/dbms/include/DB/DataStreams/NativeBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/NativeBlockOutputStream.h @@ -6,8 +6,14 @@ namespace DB { +class WriteBuffer; +class CompressedWriteBuffer; + + /** Сериализует поток блоков в родном бинарном формате (с именами и типами столбцов). * Предназначено для взаимодействия между серверами. + * + * Может быть указан поток для записи индекса. Индекс содержит смещения до каждого кусочка каждого столбца. */ class NativeBlockOutputStream : public IBlockOutputStream { @@ -15,8 +21,9 @@ public: /** В случае указания ненулевой client_revision, может записываться дополнительная информация о блоке, * в зависимости от поддерживаемой для указанной ревизии. */ - NativeBlockOutputStream(WriteBuffer & ostr_, UInt64 client_revision_ = 0) - : ostr(ostr_), client_revision(client_revision_) {} + NativeBlockOutputStream( + WriteBuffer & ostr_, UInt64 client_revision_ = 0, + WriteBuffer * index_ostr_ = nullptr); void write(const Block & block) override; void flush() override { ostr.next(); } @@ -26,6 +33,10 @@ public: private: WriteBuffer & ostr; UInt64 client_revision; + + WriteBuffer * index_ostr; + /// Если требуется записывать индекс, то ostr обязан быть CompressedWriteBuffer. + CompressedWriteBuffer * ostr_concrete = nullptr; }; } diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index 30b044c5488..da58b0b158f 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -84,6 +84,16 @@ public: if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; + { + std::lock_guard lock(external_tables_mutex); + + /// Останавливаем отправку внешних данных. + for (auto & vec : external_tables_data) + for (auto & elem : vec) + if (IProfilingBlockInputStream * stream = dynamic_cast(elem.first.get())) + stream->cancel(); + } + if (!isQueryPending() || hasThrownException()) return; @@ -101,52 +111,54 @@ public: parallel_replicas->disconnect(); } + + /// Отправляет запрос (инициирует вычисления) раньше, чем read. + void readPrefix() override + { + if (!sent_query) + sendQuery(); + } + protected: /// Отправить на удаленные реплики все временные таблицы void sendExternalTables() { size_t count = parallel_replicas->size(); - std::vector instances; - instances.reserve(count); - - for (size_t i = 0; i < count; ++i) { - ExternalTablesData res; - for (const auto & table : external_tables) + std::lock_guard lock(external_tables_mutex); + + external_tables_data.reserve(count); + + for (size_t i = 0; i < count; ++i) { - StoragePtr cur = table.second; - QueryProcessingStage::Enum stage = QueryProcessingStage::Complete; - DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings, - stage, DEFAULT_BLOCK_SIZE, 1); - if (input.size() == 0) - res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first)); - else - res.push_back(std::make_pair(input[0], table.first)); + ExternalTablesData res; + for (const auto & table : external_tables) + { + StoragePtr cur = table.second; + QueryProcessingStage::Enum stage = QueryProcessingStage::Complete; + DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings, + stage, DEFAULT_BLOCK_SIZE, 1); + if (input.size() == 0) + res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first)); + else + res.push_back(std::make_pair(input[0], table.first)); + } + external_tables_data.push_back(std::move(res)); } - instances.push_back(std::move(res)); } - parallel_replicas->sendExternalTablesData(instances); + parallel_replicas->sendExternalTablesData(external_tables_data); } Block readImpl() override { if (!sent_query) { - createParallelReplicas(); + sendQuery(); if (settings.skip_unavailable_shards && 0 == parallel_replicas->size()) - return Block(); - - established = true; - - parallel_replicas->sendQuery(query, "", stage, true); - - established = false; - sent_query = true; - - sendExternalTables(); + return {}; } while (true) @@ -267,6 +279,23 @@ protected: } private: + void sendQuery() + { + createParallelReplicas(); + + if (settings.skip_unavailable_shards && 0 == parallel_replicas->size()) + return; + + established = true; + + parallel_replicas->sendQuery(query, "", stage, true); + + established = false; + sent_query = true; + + sendExternalTables(); + } + /// ITable::read requires a Context, therefore we should create one if the user can't supply it static Context & getDefaultContext() { @@ -302,6 +331,10 @@ private: QueryProcessingStage::Enum stage; Context context; + /// Потоки для чтения из временных таблиц - для последующей отправки данных на удалённые серверы для GLOBAL-подзапросов. + std::vector external_tables_data; + std::mutex external_tables_mutex; + /// Установили соединения с репликами, но ещё не отправили запрос. std::atomic established { false }; diff --git a/dbms/include/DB/Dictionaries/CacheDictionary.h b/dbms/include/DB/Dictionaries/CacheDictionary.h index 07db7c0dda2..15bbe05cb19 100644 --- a/dbms/include/DB/Dictionaries/CacheDictionary.h +++ b/dbms/include/DB/Dictionaries/CacheDictionary.h @@ -620,9 +620,9 @@ private: mutable std::mt19937_64 rnd_engine{getSeed()}; mutable std::size_t bytes_allocated = 0; - mutable std::atomic element_count{}; - mutable std::atomic hit_count{}; - mutable std::atomic query_count{}; + mutable std::atomic element_count{0}; + mutable std::atomic hit_count{0}; + mutable std::atomic query_count{0}; const std::chrono::time_point creation_time = std::chrono::system_clock::now(); }; diff --git a/dbms/include/DB/Dictionaries/FileDictionarySource.h b/dbms/include/DB/Dictionaries/FileDictionarySource.h index 41e55f64de5..a2f9b0d3c9a 100644 --- a/dbms/include/DB/Dictionaries/FileDictionarySource.h +++ b/dbms/include/DB/Dictionaries/FileDictionarySource.h @@ -20,8 +20,7 @@ class FileDictionarySource final : public IDictionarySource public: FileDictionarySource(const std::string & filename, const std::string & format, Block & sample_block, const Context & context) - : filename{filename}, format{format}, sample_block{sample_block}, context(context), - last_modification{getLastModification()} + : filename{filename}, format{format}, sample_block{sample_block}, context(context) {} FileDictionarySource(const FileDictionarySource & other) diff --git a/dbms/include/DB/Dictionaries/FlatDictionary.h b/dbms/include/DB/Dictionaries/FlatDictionary.h index 1ae5b976fde..0c1c2fb33e5 100644 --- a/dbms/include/DB/Dictionaries/FlatDictionary.h +++ b/dbms/include/DB/Dictionaries/FlatDictionary.h @@ -20,9 +20,10 @@ class FlatDictionary final : public IDictionary { public: FlatDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime) + DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty) : name{name}, dict_struct(dict_struct), - source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime) + source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), + require_nonempty(require_nonempty) { createAttributes(); @@ -40,7 +41,7 @@ public: } FlatDictionary(const FlatDictionary & other) - : FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime} + : FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty} {} std::exception_ptr getCreationException() const override { return creation_exception; } @@ -198,6 +199,9 @@ private: } stream->readSuffix(); + + if (require_nonempty && 0 == element_count) + throw Exception("Dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY); } template @@ -348,6 +352,7 @@ private: const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; + const bool require_nonempty; std::map attribute_index_by_name; std::vector attributes; @@ -356,7 +361,7 @@ private: std::size_t bytes_allocated = 0; std::size_t element_count = 0; std::size_t bucket_count = 0; - mutable std::atomic query_count; + mutable std::atomic query_count{0}; std::chrono::time_point creation_time; diff --git a/dbms/include/DB/Dictionaries/HashedDictionary.h b/dbms/include/DB/Dictionaries/HashedDictionary.h index 08aad57d63a..a73f8869f81 100644 --- a/dbms/include/DB/Dictionaries/HashedDictionary.h +++ b/dbms/include/DB/Dictionaries/HashedDictionary.h @@ -18,9 +18,10 @@ class HashedDictionary final : public IDictionary { public: HashedDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime) + DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty) : name{name}, dict_struct(dict_struct), - source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime) + source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), + require_nonempty(require_nonempty) { createAttributes(); @@ -38,7 +39,7 @@ public: } HashedDictionary(const HashedDictionary & other) - : HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime} + : HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty} {} std::exception_ptr getCreationException() const override { return creation_exception; } @@ -196,6 +197,9 @@ private: } stream->readSuffix(); + + if (require_nonempty && 0 == element_count) + throw Exception("Dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY); } template @@ -334,6 +338,7 @@ private: const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; + const bool require_nonempty; std::map attribute_index_by_name; std::vector attributes; @@ -342,7 +347,7 @@ private: std::size_t bytes_allocated = 0; std::size_t element_count = 0; std::size_t bucket_count = 0; - mutable std::atomic query_count{}; + mutable std::atomic query_count{0}; std::chrono::time_point creation_time; diff --git a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h index c0ee3974d49..abf58b2767f 100644 --- a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h +++ b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h @@ -23,10 +23,10 @@ public: db{config.getString(config_prefix + ".db", "")}, table{config.getString(config_prefix + ".table")}, where{config.getString(config_prefix + ".where", "")}, + dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}, sample_block{sample_block}, pool{config, config_prefix}, - load_all_query{composeLoadAllQuery()}, - last_modification{getLastModification()} + load_all_query{composeLoadAllQuery()} {} /// copy-constructor is provided in order to support cloneability @@ -35,6 +35,7 @@ public: db{other.db}, table{other.table}, where{other.where}, + dont_check_update_time{other.dont_check_update_time}, sample_block{other.sample_block}, pool{other.pool}, load_all_query{other.load_all_query}, last_modification{other.last_modification} @@ -43,18 +44,27 @@ public: BlockInputStreamPtr loadAll() override { last_modification = getLastModification(); + + LOG_TRACE(log, load_all_query); return new MySQLBlockInputStream{pool.Get(), load_all_query, sample_block, max_block_size}; } BlockInputStreamPtr loadIds(const std::vector & ids) override { - last_modification = getLastModification(); - const auto query = composeLoadIdsQuery(ids); + /// Здесь не логгируем и не обновляем время модификации, так как запрос может быть большим, и часто задаваться. + const auto query = composeLoadIdsQuery(ids); return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size}; } - bool isModified() const override { return getLastModification() > last_modification; } + bool isModified() const override + { + if (dont_check_update_time) + return true; + + return getLastModification() > last_modification; + } + bool supportsSelectiveLoad() const override { return true; } DictionarySourcePtr clone() const override { return std::make_unique(*this); } @@ -65,27 +75,47 @@ public: } private: + Logger * log = &Logger::get("MySQLDictionarySource"); + mysqlxx::DateTime getLastModification() const { - const auto Update_time_idx = 12; mysqlxx::DateTime update_time{std::time(nullptr)}; + if (dont_check_update_time) + return update_time; + try { auto connection = pool.Get(); - auto query = connection->query("SHOW TABLE STATUS LIKE '%" + strconvert::escaped_for_like(table) + "%';"); + auto query = connection->query("SHOW TABLE STATUS LIKE '" + strconvert::escaped_for_like(table) + "'"); + + LOG_TRACE(log, query.str()); + auto result = query.use(); + size_t fetched_rows = 0; if (auto row = result.fetch()) { - const auto & update_time_value = row[Update_time_idx]; + ++fetched_rows; + const auto UPDATE_TIME_IDX = 12; + const auto & update_time_value = row[UPDATE_TIME_IDX]; if (!update_time_value.isNull()) + { update_time = update_time_value.getDateTime(); + LOG_TRACE(log, "Got update time: " << update_time); + } /// fetch remaining rows to avoid "commands out of sync" error - while (auto row = result.fetch()); + while (auto row = result.fetch()) + ++fetched_rows; } + + if (0 == fetched_rows) + LOG_ERROR(log, "Cannot find table in SHOW TABLE STATUS result."); + + if (fetched_rows > 1) + LOG_ERROR(log, "Found more than one table in SHOW TABLE STATUS result."); } catch (...) { @@ -209,6 +239,7 @@ private: const std::string db; const std::string table; const std::string where; + const bool dont_check_update_time; Block sample_block; mutable mysqlxx::PoolWithFailover pool; const std::string load_all_query; diff --git a/dbms/include/DB/Dictionaries/RangeHashedDictionary.h b/dbms/include/DB/Dictionaries/RangeHashedDictionary.h index 0435baecc93..888173b8940 100644 --- a/dbms/include/DB/Dictionaries/RangeHashedDictionary.h +++ b/dbms/include/DB/Dictionaries/RangeHashedDictionary.h @@ -19,9 +19,10 @@ class RangeHashedDictionary final : public IDictionaryBase public: RangeHashedDictionary( const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr, - const DictionaryLifetime dict_lifetime) + const DictionaryLifetime dict_lifetime, bool require_nonempty) : name{name}, dict_struct(dict_struct), - source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime) + source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), + require_nonempty(require_nonempty) { createAttributes(); @@ -39,7 +40,7 @@ public: } RangeHashedDictionary(const RangeHashedDictionary & other) - : RangeHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime} + : RangeHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty} {} std::exception_ptr getCreationException() const override { return creation_exception; } @@ -218,6 +219,9 @@ private: } stream->readSuffix(); + + if (require_nonempty && 0 == element_count) + throw Exception("Dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY); } template @@ -410,6 +414,7 @@ private: const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; + const bool require_nonempty; std::map attribute_index_by_name; std::vector attributes; @@ -417,7 +422,7 @@ private: std::size_t bytes_allocated = 0; std::size_t element_count = 0; std::size_t bucket_count = 0; - mutable std::atomic query_count{}; + mutable std::atomic query_count{0}; std::chrono::time_point creation_time; diff --git a/dbms/include/DB/Functions/FunctionsArray.h b/dbms/include/DB/Functions/FunctionsArray.h index 498af1955b6..6d1be3161e6 100644 --- a/dbms/include/DB/Functions/FunctionsArray.h +++ b/dbms/include/DB/Functions/FunctionsArray.h @@ -43,6 +43,8 @@ namespace DB * Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1] * arrayEnumerateUniq(arr1, arr2...) * - для кортежей из элементов на соответствующих позициях в нескольких массивах. + * + * emptyArrayToSingle(arr) - заменить пустые массивы на массивы из одного элемента со значением "по-умолчанию". */ @@ -1695,13 +1697,263 @@ private: }; +class FunctionEmptyArrayToSingle : public IFunction +{ +public: + static constexpr auto name = "emptyArrayToSingle"; + static IFunction * create(const Context & context) { return new FunctionEmptyArrayToSingle; } + + /// Получить имя функции. + String getName() const + { + return name; + } + + /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. + DataTypePtr getReturnType(const DataTypes & arguments) 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); + + const DataTypeArray * array_type = typeid_cast(arguments[0].get()); + if (!array_type) + throw Exception("Argument for function " + getName() + " must be array.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]->clone(); + } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (executeConst(block, arguments, result)) + return; + + const ColumnArray * array = typeid_cast(block.getByPosition(arguments[0]).column.get()); + if (!array) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + ColumnPtr res_ptr = array->cloneEmpty(); + block.getByPosition(result).column = res_ptr; + ColumnArray & res = static_cast(*res_ptr); + + const IColumn & src_data = array->getData(); + const ColumnArray::Offsets_t & src_offsets = array->getOffsets(); + IColumn & res_data = res.getData(); + ColumnArray::Offsets_t & res_offsets = res.getOffsets(); + + if (!( executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeNumber (src_data, src_offsets, res_data, res_offsets) + || executeString (src_data, src_offsets, res_data, res_offsets) + || executeFixedString (src_data, src_offsets, res_data, res_offsets))) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result) + { + if (const ColumnConstArray * const_array = typeid_cast(block.getByPosition(arguments[0]).column.get())) + { + if (const_array->getData().empty()) + { + auto nested_type = typeid_cast(*block.getByPosition(arguments[0]).type).getNestedType(); + + block.getByPosition(result).column = new ColumnConstArray( + block.rowsInFirstColumn(), + {nested_type->getDefault()}, + nested_type->clone()); + } + else + block.getByPosition(result).column = block.getByPosition(arguments[0]).column; + + return true; + } + else + return false; + } + + template + bool executeNumber( + const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets, + IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets) + { + if (const ColumnVector * src_data_concrete = typeid_cast *>(&src_data)) + { + const PODArray & src_data = src_data_concrete->getData(); + PODArray & res_data = typeid_cast &>(res_data_col).getData(); + size_t size = src_offsets.size(); + res_offsets.resize(size); + res_data.reserve(src_data.size()); + + ColumnArray::Offset_t src_prev_offset = 0; + ColumnArray::Offset_t res_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_offsets[i] != src_prev_offset) + { + size_t size_to_write = src_offsets[i] - src_prev_offset; + size_t prev_res_data_size = res_data.size(); + res_data.resize(prev_res_data_size + size_to_write); + memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset], size_to_write * sizeof(T)); + res_prev_offset += size_to_write; + res_offsets[i] = res_prev_offset; + } + else + { + res_data.push_back(T()); + ++res_prev_offset; + res_offsets[i] = res_prev_offset; + } + + src_prev_offset = src_offsets[i]; + } + + return true; + } + else + return false; + } + + bool executeFixedString( + const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets, + IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets) + { + if (const ColumnFixedString * src_data_concrete = typeid_cast(&src_data)) + { + const size_t n = src_data_concrete->getN(); + const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); + ColumnFixedString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); + size_t size = src_offsets.size(); + res_offsets.resize(size); + res_data.reserve(src_data.size()); + + ColumnArray::Offset_t src_prev_offset = 0; + ColumnArray::Offset_t res_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_offsets[i] != src_prev_offset) + { + size_t size_to_write = src_offsets[i] - src_prev_offset; + size_t prev_res_data_size = res_data.size(); + res_data.resize(prev_res_data_size + size_to_write * n); + memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset], size_to_write * n); + res_prev_offset += size_to_write; + res_offsets[i] = res_prev_offset; + } + else + { + size_t prev_res_data_size = res_data.size(); + res_data.resize(prev_res_data_size + n); + memset(&res_data[prev_res_data_size], 0, n); + ++res_prev_offset; + res_offsets[i] = res_prev_offset; + } + + src_prev_offset = src_offsets[i]; + } + + return true; + } + else + return false; + } + + bool executeString( + const IColumn & src_data, const ColumnArray::Offsets_t & src_array_offsets, + IColumn & res_data_col, ColumnArray::Offsets_t & res_array_offsets) + { + if (const ColumnString * src_data_concrete = typeid_cast(&src_data)) + { + const ColumnString::Offsets_t & src_string_offsets = src_data_concrete->getOffsets(); + ColumnString::Offsets_t & res_string_offsets = typeid_cast(res_data_col).getOffsets(); + + const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); + ColumnString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); + + size_t size = src_array_offsets.size(); + res_array_offsets.resize(size); + res_string_offsets.reserve(src_string_offsets.size()); + res_data.reserve(src_data.size()); + + ColumnArray::Offset_t src_array_prev_offset = 0; + ColumnArray::Offset_t res_array_prev_offset = 0; + + ColumnString::Offset_t src_string_prev_offset = 0; + ColumnString::Offset_t res_string_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + if (src_array_offsets[i] != src_array_prev_offset) + { + size_t array_size = src_array_offsets[i] - src_array_prev_offset; + + size_t bytes_to_copy = 0; + size_t from_string_prev_offset_local = src_string_prev_offset; + for (size_t j = 0; j < array_size; ++j) + { + size_t string_size = src_string_offsets[src_array_prev_offset + j] - from_string_prev_offset_local; + + res_string_prev_offset += string_size; + res_string_offsets.push_back(res_string_prev_offset); + + from_string_prev_offset_local += string_size; + bytes_to_copy += string_size; + } + + size_t res_data_old_size = res_data.size(); + res_data.resize(res_data_old_size + bytes_to_copy); + memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy); + + res_array_prev_offset += array_size; + res_array_offsets[i] = res_array_prev_offset; + } + else + { + res_data.push_back(0); /// Пустая строка, включая ноль на конце. + + ++res_string_prev_offset; + res_string_offsets.push_back(res_string_prev_offset); + + ++res_array_prev_offset; + res_array_offsets[i] = res_array_prev_offset; + } + + src_array_prev_offset = src_array_offsets[i]; + + if (src_array_prev_offset) + src_string_prev_offset = src_string_offsets[src_array_prev_offset - 1]; + } + + return true; + } + else + return false; + } +}; + + struct NameHas { static constexpr auto name = "has"; }; struct NameIndexOf { static constexpr auto name = "indexOf"; }; struct NameCountEqual { static constexpr auto name = "countEqual"; }; -typedef FunctionArrayIndex FunctionHas; +typedef FunctionArrayIndex FunctionHas; typedef FunctionArrayIndex FunctionIndexOf; -typedef FunctionArrayIndex FunctionCountEqual; +typedef FunctionArrayIndex FunctionCountEqual; using FunctionEmptyArrayUInt8 = FunctionEmptyArray; using FunctionEmptyArrayUInt16 = FunctionEmptyArray; diff --git a/dbms/include/DB/Functions/FunctionsDictionaries.h b/dbms/include/DB/Functions/FunctionsDictionaries.h index d49a8ed8180..cf7eaea60ff 100644 --- a/dbms/include/DB/Functions/FunctionsDictionaries.h +++ b/dbms/include/DB/Functions/FunctionsDictionaries.h @@ -862,7 +862,7 @@ private: dict->getString(attr_name, ids, out.get()); block.getByPosition(result).column = new ColumnConst{ - id_col->size(), out->getDataAtWithTerminatingZero(0).toString() + id_col->size(), out->getDataAt(0).toString() }; } else @@ -967,7 +967,7 @@ private: dictionary->getString(attr_name, ids, dates, out.get()); block.getByPosition(result).column = new ColumnConst{ - id_col->size(), out->getDataAtWithTerminatingZero(0).toString() + id_col->size(), out->getDataAt(0).toString() }; } else diff --git a/dbms/include/DB/Functions/FunctionsURL.h b/dbms/include/DB/Functions/FunctionsURL.h index 437849fba30..6a588834786 100644 --- a/dbms/include/DB/Functions/FunctionsURL.h +++ b/dbms/include/DB/Functions/FunctionsURL.h @@ -404,31 +404,36 @@ struct ExtractURLParameterImpl { size_t cur_offset = offsets[i]; + const char * str = reinterpret_cast(&data[prev_offset]); + const char * pos = nullptr; - - do + const char * begin = strchr(str, '?'); + if (begin != nullptr) { - const char * str = reinterpret_cast(&data[prev_offset]); - - const char * begin = strchr(str, '?'); - if (begin == nullptr) - break; - - pos = strstr(begin + 1, param_str); - if (pos == nullptr) - break; - if (pos != begin + 1 && *(pos - 1) != ';' && *(pos - 1) != '&') + pos = begin + 1; + while (true) { - pos = nullptr; - break; - } + pos = strstr(pos, param_str); - pos += param_len; - } while (false); + if (pos == nullptr) + break; + + if (pos[-1] != '?' && pos[-1] != '&') + { + pos += param_len; + continue; + } + else + { + pos += param_len; + break; + } + } + } if (pos != nullptr) { - const char * end = strpbrk(pos, "&;#"); + const char * end = strpbrk(pos, "&#"); if (end == nullptr) end = pos + strlen(pos); diff --git a/dbms/include/DB/IO/BufferWithOwnMemory.h b/dbms/include/DB/IO/BufferWithOwnMemory.h index 9c63e592e94..96aa8728428 100644 --- a/dbms/include/DB/IO/BufferWithOwnMemory.h +++ b/dbms/include/DB/IO/BufferWithOwnMemory.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include @@ -18,7 +18,7 @@ namespace DB * Отличается тем, что не делает лишний memset. (И почти ничего не делает.) * Также можно попросить выделять выровненный кусок памяти. */ -struct Memory : boost::noncopyable +struct Memory : boost::noncopyable, Allocator { size_t m_capacity = 0; size_t m_size = 0; @@ -66,16 +66,22 @@ struct Memory : boost::noncopyable } else { - dealloc(); - + new_size = align(new_size); + m_data = reinterpret_cast(Allocator::realloc(m_data, m_capacity, new_size, alignment)); m_capacity = new_size; m_size = m_capacity; - - alloc(); } } private: + size_t align(size_t value) const + { + if (!alignment) + return value; + + return (value + alignment - 1) / alignment * alignment; + } + void alloc() { if (!m_capacity) @@ -87,33 +93,10 @@ private: ProfileEvents::increment(ProfileEvents::IOBufferAllocs); ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, m_capacity); - if (current_memory_tracker) - current_memory_tracker->alloc(m_capacity); - - char * new_m_data = nullptr; - - if (!alignment) - { - new_m_data = reinterpret_cast(malloc(m_capacity)); - - if (!new_m_data) - throw Exception("Cannot allocate memory (malloc)", ErrorCodes::CANNOT_ALLOCATE_MEMORY); - - m_data = new_m_data; - - return; - } - - size_t aligned_capacity = (m_capacity + alignment - 1) / alignment * alignment; - m_capacity = aligned_capacity; + size_t new_capacity = align(m_capacity); + m_data = reinterpret_cast(Allocator::alloc(new_capacity, alignment)); + m_capacity = new_capacity; m_size = m_capacity; - - int res = posix_memalign(reinterpret_cast(&new_m_data), alignment, m_capacity); - - if (0 != res) - DB::throwFromErrno("Cannot allocate memory (posix_memalign)", ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); - - m_data = new_m_data; } void dealloc() @@ -121,11 +104,8 @@ private: if (!m_data) return; - free(reinterpret_cast(m_data)); + Allocator::free(reinterpret_cast(m_data), m_capacity); m_data = nullptr; /// Чтобы избежать double free, если последующий вызов alloc кинет исключение. - - if (current_memory_tracker) - current_memory_tracker->free(m_capacity); } }; diff --git a/dbms/include/DB/IO/HashingWriteBuffer.h b/dbms/include/DB/IO/HashingWriteBuffer.h index 50f26f65e5a..154debf418a 100644 --- a/dbms/include/DB/IO/HashingWriteBuffer.h +++ b/dbms/include/DB/IO/HashingWriteBuffer.h @@ -15,8 +15,8 @@ template class IHashingBuffer : public BufferWithOwnMemory { public: - IHashingBuffer(size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE) : - block_pos(0), block_size(block_size_), state(0, 0) + IHashingBuffer(size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE) + : BufferWithOwnMemory(block_size_), block_pos(0), block_size(block_size_), state(0, 0) { } diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index 5459c06b82b..65fbdce1900 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -674,17 +674,6 @@ typedef SharedPtr AggregatedDataVariantsPtr; typedef std::vector ManyAggregatedDataVariants; -/** Достать вариант агрегации по его типу. */ -template Method & getDataVariant(AggregatedDataVariants & variants); - -#define M(NAME, IS_TWO_LEVEL) \ - template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant(AggregatedDataVariants & variants) { return *variants.NAME; } - -APPLY_FOR_AGGREGATED_VARIANTS(M) - -#undef M - - /** Агрегирует источник блоков. */ class Aggregator @@ -733,11 +722,15 @@ public: */ AggregatedDataVariantsPtr merge(ManyAggregatedDataVariants & data_variants, size_t max_threads); - /** Объединить несколько агрегированных блоков в одну структуру данных. + /** Объединить поток частично агрегированных блоков в одну структуру данных. * (Доагрегировать несколько блоков, которые представляют собой результат независимых агрегаций с удалённых серверов.) */ void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads); + /** Объединить несколько частично агрегированных блоков в один. + */ + Block mergeBlocks(BlocksList & blocks, bool final); + using CancellationHook = std::function; /** Установить функцию, которая проверяет, можно ли прервать текущую задачу. @@ -974,4 +967,15 @@ protected: }; +/** Достать вариант агрегации по его типу. */ +template Method & getDataVariant(AggregatedDataVariants & variants); + +#define M(NAME, IS_TWO_LEVEL) \ + template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant(AggregatedDataVariants & variants) { return *variants.NAME; } + +APPLY_FOR_AGGREGATED_VARIANTS(M) + +#undef M + + } diff --git a/dbms/include/DB/Interpreters/ExpressionActions.h b/dbms/include/DB/Interpreters/ExpressionActions.h index b4dedade097..39aed32e3c3 100644 --- a/dbms/include/DB/Interpreters/ExpressionActions.h +++ b/dbms/include/DB/Interpreters/ExpressionActions.h @@ -66,6 +66,7 @@ public: /// Для ARRAY_JOIN NameSet array_joined_columns; + bool array_join_is_left; /// Для JOIN const Join * join = nullptr; @@ -122,13 +123,14 @@ public: return a; } - static ExpressionAction arrayJoin(const NameSet & array_joined_columns) + static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left) { if (array_joined_columns.empty()) throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR); ExpressionAction a; a.type = ARRAY_JOIN; a.array_joined_columns = array_joined_columns; + a.array_join_is_left = array_join_is_left; return a; } diff --git a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h index 086cacdae18..36d7ea25e5e 100644 --- a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h +++ b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h @@ -226,6 +226,10 @@ private: /// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn. void makeSet(ASTFunction * node, const Block & sample_block); + /// Замена скалярных подзапросов на значения-константы. + void executeScalarSubqueries(); + void executeScalarSubqueriesImpl(ASTPtr & ast); + /// Находит глобальные подзапросы в секциях GLOBAL IN/JOIN. Заполняет external_tables. void initGlobalSubqueriesAndExternalTables(); void initGlobalSubqueries(ASTPtr & ast); diff --git a/dbms/include/DB/Interpreters/ExternalDictionaries.h b/dbms/include/DB/Interpreters/ExternalDictionaries.h index f3747a52cd6..cf2fbe36f10 100644 --- a/dbms/include/DB/Interpreters/ExternalDictionaries.h +++ b/dbms/include/DB/Interpreters/ExternalDictionaries.h @@ -57,9 +57,19 @@ private: std::uint64_t error_count; }; + /** Имя словаря -> словарь. + */ std::unordered_map dictionaries; - std::unordered_map update_times; + + /** Здесь находятся словари, которых ещё ни разу не удалось загрузить. + * В dictionaries они тоже присутствуют, но с нулевым указателем dict. + */ std::unordered_map failed_dictionaries; + + /** И для обычных и для failed_dictionaries. + */ + std::unordered_map update_times; + std::mt19937_64 rnd_engine{getSeed()}; Context & context; diff --git a/dbms/include/DB/Interpreters/InterpreterCreateQuery.h b/dbms/include/DB/Interpreters/InterpreterCreateQuery.h index 7382b797147..367253edcb2 100644 --- a/dbms/include/DB/Interpreters/InterpreterCreateQuery.h +++ b/dbms/include/DB/Interpreters/InterpreterCreateQuery.h @@ -24,8 +24,7 @@ public: */ BlockIO execute() override { - executeImpl(false); - return {}; + return executeImpl(false); } /** assume_metadata_exists - не проверять наличие файла с метаданными и не создавать его @@ -45,7 +44,7 @@ public: const ColumnDefaults & column_defaults); private: - void executeImpl(bool assume_metadata_exists); + BlockIO executeImpl(bool assume_metadata_exists); /// AST в список столбцов с типами. Столбцы типа Nested развернуты в список настоящих столбцов. using ColumnsAndDefaults = std::pair; diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 2bc4c0bca4a..36d4fc9db6a 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -91,6 +91,8 @@ struct Settings M(SettingUInt64, min_count_to_compile, 3) \ /** При каком количестве ключей, начинает использоваться двухуровневая агрегация. 0 - никогда не использовать. */ \ M(SettingUInt64, group_by_two_level_threshold, 100000) \ + /** Включён ли экономный по памяти режим распределённой агрегации. */ \ + M(SettingBool, distributed_aggregation_memory_efficient, false) \ \ /** Максимальное количество используемых реплик каждого шарда при выполнении запроса */ \ M(SettingUInt64, max_parallel_replicas, 1) \ diff --git a/dbms/include/DB/Interpreters/sortBlock.h b/dbms/include/DB/Interpreters/sortBlock.h index 4b9982bd603..9026611be5e 100644 --- a/dbms/include/DB/Interpreters/sortBlock.h +++ b/dbms/include/DB/Interpreters/sortBlock.h @@ -18,4 +18,9 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit */ void stableSortBlock(Block & block, const SortDescription & description); +/** То же, что и stableSortBlock, но не сортировать блок, а только рассчитать перестановку значений, + * чтобы потом можно было переставить значения столбцов самостоятельно. + */ +void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation); + } diff --git a/dbms/include/DB/Parsers/ASTAlterQuery.h b/dbms/include/DB/Parsers/ASTAlterQuery.h index 1d22d25e186..d0e88080071 100644 --- a/dbms/include/DB/Parsers/ASTAlterQuery.h +++ b/dbms/include/DB/Parsers/ASTAlterQuery.h @@ -1,6 +1,8 @@ #pragma once #include +#include + namespace DB { @@ -98,5 +100,87 @@ public: } return res; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + frame.need_parens = false; + + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); + + if (!table.empty()) + { + if (!database.empty()) + { + settings.ostr << indent_str << database; + settings.ostr << "."; + } + settings.ostr << indent_str << table; + } + settings.ostr << settings.nl_or_ws; + + for (size_t i = 0; i < parameters.size(); ++i) + { + const ASTAlterQuery::Parameters & p = parameters[i]; + + if (p.type == ASTAlterQuery::ADD_COLUMN) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (settings.hilite ? hilite_none : ""); + p.col_decl->formatImpl(settings, state, frame); + + /// AFTER + if (p.column) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); + p.column->formatImpl(settings, state, frame); + } + } + else if (p.type == ASTAlterQuery::DROP_COLUMN) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP COLUMN " << (settings.hilite ? hilite_none : ""); + p.column->formatImpl(settings, state, frame); + } + else if (p.type == ASTAlterQuery::MODIFY_COLUMN) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : ""); + p.col_decl->formatImpl(settings, state, frame); + } + else if (p.type == ASTAlterQuery::DROP_PARTITION) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (p.detach ? "DETACH" : "DROP") << " PARTITION " + << (settings.hilite ? hilite_none : ""); + p.partition->formatImpl(settings, state, frame); + } + else if (p.type == ASTAlterQuery::ATTACH_PARTITION) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ATTACH " << (p.unreplicated ? "UNREPLICATED " : "") + << (p.part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : ""); + p.partition->formatImpl(settings, state, frame); + } + else if (p.type == ASTAlterQuery::FETCH_PARTITION) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FETCH " << (p.unreplicated ? "UNREPLICATED " : "") + << "PARTITION " << (settings.hilite ? hilite_none : ""); + p.partition->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " FROM " << (settings.hilite ? hilite_none : "") << mysqlxx::quote << p.from; + } + else if (p.type == ASTAlterQuery::FREEZE_PARTITION) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FREEZE PARTITION " << (settings.hilite ? hilite_none : ""); + p.partition->formatImpl(settings, state, frame); + } + else + throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + + std::string comma = (i < (parameters.size() -1) ) ? "," : ""; + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << comma << (settings.hilite ? hilite_none : ""); + + settings.ostr << settings.nl_or_ws; + } + } }; + } diff --git a/dbms/include/DB/Parsers/ASTAsterisk.h b/dbms/include/DB/Parsers/ASTAsterisk.h index aa90d676c71..f0741539267 100644 --- a/dbms/include/DB/Parsers/ASTAsterisk.h +++ b/dbms/include/DB/Parsers/ASTAsterisk.h @@ -16,6 +16,12 @@ public: String getID() const override { return "Asterisk"; } ASTPtr clone() const override { return new ASTAsterisk(*this); } String getColumnName() const override { return "*"; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << "*"; + } }; } diff --git a/dbms/include/DB/Parsers/ASTCheckQuery.h b/dbms/include/DB/Parsers/ASTCheckQuery.h index 901ad7ef567..74b64aeaef8 100644 --- a/dbms/include/DB/Parsers/ASTCheckQuery.h +++ b/dbms/include/DB/Parsers/ASTCheckQuery.h @@ -19,6 +19,28 @@ struct ASTCheckQuery : public IAST std::string database; std::string table; + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + std::string nl_or_nothing = settings.one_line ? "" : "\n"; + + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + std::string nl_or_ws = settings.one_line ? " " : "\n"; + + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (settings.hilite ? hilite_none : ""); + + if (!table.empty()) + { + if (!database.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << database << (settings.hilite ? hilite_none : ""); + settings.ostr << "."; + } + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << table << (settings.hilite ? hilite_none : ""); + } + settings.ostr << nl_or_ws; + } }; } diff --git a/dbms/include/DB/Parsers/ASTColumnDeclaration.h b/dbms/include/DB/Parsers/ASTColumnDeclaration.h index 9862c5a81b6..722bc6d8283 100644 --- a/dbms/include/DB/Parsers/ASTColumnDeclaration.h +++ b/dbms/include/DB/Parsers/ASTColumnDeclaration.h @@ -40,6 +40,26 @@ public: return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + frame.need_parens = false; + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + + settings.ostr << settings.nl_or_ws << indent_str << backQuoteIfNeed(name); + if (type) + { + settings.ostr << ' '; + type->formatImpl(settings, state, frame); + } + + if (default_expression) + { + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : "") << ' '; + default_expression->formatImpl(settings, state, frame); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTCreateQuery.h b/dbms/include/DB/Parsers/ASTCreateQuery.h index 33d69a8e8e3..b76cfedc2d2 100644 --- a/dbms/include/DB/Parsers/ASTCreateQuery.h +++ b/dbms/include/DB/Parsers/ASTCreateQuery.h @@ -30,7 +30,7 @@ public: ASTCreateQuery() = default; ASTCreateQuery(const StringRange range_) : IAST(range_) {} - + /** Получить текст, который идентифицирует этот элемент. */ String getID() const override { return (attach ? "AttachQuery_" : "CreateQuery_") + database + "_" + table; }; @@ -48,6 +48,74 @@ public: return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + frame.need_parens = false; + + if (!database.empty() && table.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH DATABASE " : "CREATE DATABASE ") << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(database); + return; + } + + { + std::string what = "TABLE"; + if (is_view) + what = "VIEW"; + if (is_materialized_view) + what = "MATERIALIZED VIEW"; + + settings.ostr + << (settings.hilite ? hilite_keyword : "") + << (attach ? "ATTACH " : "CREATE ") + << (is_temporary ? "TEMPORARY " : "") + << what + << " " << (if_not_exists ? "IF NOT EXISTS " : "") + << (settings.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + } + + if (!as_table.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "") + << (!as_database.empty() ? backQuoteIfNeed(as_database) + "." : "") << backQuoteIfNeed(as_table); + } + + if (columns) + { + settings.ostr << (settings.one_line ? " (" : "\n("); + FormatStateStacked frame_nested = frame; + ++frame_nested.indent; + columns->formatImpl(settings, state, frame_nested); + settings.ostr << (settings.one_line ? ")" : "\n)"); + } + + if (storage && !is_materialized_view && !is_view) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " ENGINE" << (settings.hilite ? hilite_none : "") << " = "; + storage->formatImpl(settings, state, frame); + } + + if (inner_storage) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " ENGINE" << (settings.hilite ? hilite_none : "") << " = "; + inner_storage->formatImpl(settings, state, frame); + } + + if (is_populate) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : ""); + } + + if (select) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); + select->formatImpl(settings, state, frame); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTDropQuery.h b/dbms/include/DB/Parsers/ASTDropQuery.h index c0ac24017d0..897f9afd4c6 100644 --- a/dbms/include/DB/Parsers/ASTDropQuery.h +++ b/dbms/include/DB/Parsers/ASTDropQuery.h @@ -24,6 +24,25 @@ public: String getID() const override { return (detach ? "DetachQuery_" : "DropQuery_") + database + "_" + table; }; ASTPtr clone() const override { return new ASTDropQuery(*this); } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + if (table.empty() && !database.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") + << (detach ? "DETACH DATABASE " : "DROP DATABASE ") + << (if_exists ? "IF EXISTS " : "") + << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(database); + return; + } + + settings.ostr << (settings.hilite ? hilite_keyword : "") + << (detach ? "DETACH TABLE " : "DROP TABLE ") + << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + } }; } diff --git a/dbms/include/DB/Parsers/ASTExpressionList.h b/dbms/include/DB/Parsers/ASTExpressionList.h index 1ec814a8d1b..1a2fdb19cb7 100644 --- a/dbms/include/DB/Parsers/ASTExpressionList.h +++ b/dbms/include/DB/Parsers/ASTExpressionList.h @@ -31,6 +31,40 @@ public: return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + { + if (it != children.begin()) + settings.ostr << ", "; + + (*it)->formatImpl(settings, state, frame); + } + } + + + friend class ASTSelectQuery; + + /** Вывести список выражений в секциях запроса SELECT - по одному выражению на строку. + */ + void formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const + { + std::string indent_str = "\n" + std::string(4 * (frame.indent + 1), ' '); + + ++frame.indent; + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + { + if (it != children.begin()) + settings.ostr << ", "; + + if (children.size() > 1) + settings.ostr << indent_str; + + (*it)->formatImpl(settings, state, frame); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTFunction.h b/dbms/include/DB/Parsers/ASTFunction.h index 70380b67e7e..74b72b25f8b 100644 --- a/dbms/include/DB/Parsers/ASTFunction.h +++ b/dbms/include/DB/Parsers/ASTFunction.h @@ -82,6 +82,9 @@ public: return ptr; } + +protected: + void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; diff --git a/dbms/include/DB/Parsers/ASTIdentifier.h b/dbms/include/DB/Parsers/ASTIdentifier.h index 58ae38ca434..9056e7dec58 100644 --- a/dbms/include/DB/Parsers/ASTIdentifier.h +++ b/dbms/include/DB/Parsers/ASTIdentifier.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -41,6 +42,18 @@ public: { set.insert(name); } + +protected: + void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << (settings.hilite ? hilite_identifier : ""); + + WriteBufferFromOStream wb(settings.ostr, 32); + writeProbablyBackQuotedString(name, wb); + wb.next(); + + settings.ostr << (settings.hilite ? hilite_none : ""); + } }; } diff --git a/dbms/include/DB/Parsers/ASTInsertQuery.h b/dbms/include/DB/Parsers/ASTInsertQuery.h index 5e6988bcfc6..c7a1879b55d 100644 --- a/dbms/include/DB/Parsers/ASTInsertQuery.h +++ b/dbms/include/DB/Parsers/ASTInsertQuery.h @@ -42,6 +42,43 @@ public: return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + frame.need_parens = false; + + settings.ostr << (settings.hilite ? hilite_keyword : "") << "INSERT INTO " << (settings.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + + if (!insert_id.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " ID = " << (settings.hilite ? hilite_none : "") + << mysqlxx::quote << insert_id; + + if (columns) + { + settings.ostr << " ("; + columns->formatImpl(settings, state, frame); + settings.ostr << ")"; + } + + if (select) + { + settings.ostr << " "; + select->formatImpl(settings, state, frame); + } + else + { + if (!format.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORMAT " << (settings.hilite ? hilite_none : "") << format; + } + else + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " VALUES" << (settings.hilite ? hilite_none : ""); + } + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTJoin.h b/dbms/include/DB/Parsers/ASTJoin.h index 96f161c5c60..34eaeecd5e5 100644 --- a/dbms/include/DB/Parsers/ASTJoin.h +++ b/dbms/include/DB/Parsers/ASTJoin.h @@ -83,6 +83,37 @@ public: return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + frame.need_parens = false; + + settings.ostr << (settings.hilite ? hilite_keyword : ""); + + if (locality == ASTJoin::Global) + settings.ostr << "GLOBAL "; + + if (kind != ASTJoin::Cross) + settings.ostr << (strictness == ASTJoin::Any ? "ANY " : "ALL "); + + settings.ostr << (kind == ASTJoin::Inner ? "INNER " + : (kind == ASTJoin::Left ? "LEFT " + : (kind == ASTJoin::Right ? "RIGHT " + : (kind == ASTJoin::Cross ? "CROSS " + : "FULL OUTER ")))); + + settings.ostr << "JOIN " + << (settings.hilite ? hilite_none : ""); + + table->formatImpl(settings, state, frame); + + if (kind != ASTJoin::Cross) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " USING " << (settings.hilite ? hilite_none : ""); + using_expr_list->formatImpl(settings, state, frame); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTLiteral.h b/dbms/include/DB/Parsers/ASTLiteral.h index 2a610255be8..17c5d87fd4e 100644 --- a/dbms/include/DB/Parsers/ASTLiteral.h +++ b/dbms/include/DB/Parsers/ASTLiteral.h @@ -26,6 +26,12 @@ public: String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); } ASTPtr clone() const override { return new ASTLiteral(*this); } + +protected: + void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << apply_visitor(FieldVisitorToString(), value); + } }; } diff --git a/dbms/include/DB/Parsers/ASTNameTypePair.h b/dbms/include/DB/Parsers/ASTNameTypePair.h index 176c431cafa..4768c7b67a2 100644 --- a/dbms/include/DB/Parsers/ASTNameTypePair.h +++ b/dbms/include/DB/Parsers/ASTNameTypePair.h @@ -34,6 +34,15 @@ public: return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + + settings.ostr << settings.nl_or_ws << indent_str << backQuoteIfNeed(name) << " "; + type->formatImpl(settings, state, frame); + } }; } diff --git a/dbms/include/DB/Parsers/ASTOptimizeQuery.h b/dbms/include/DB/Parsers/ASTOptimizeQuery.h index 906b3d1edb5..2fb6921d2ed 100644 --- a/dbms/include/DB/Parsers/ASTOptimizeQuery.h +++ b/dbms/include/DB/Parsers/ASTOptimizeQuery.h @@ -22,6 +22,13 @@ public: String getID() const override { return "OptimizeQuery_" + database + "_" + table; }; ASTPtr clone() const override { return new ASTOptimizeQuery(*this); } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + } }; } diff --git a/dbms/include/DB/Parsers/ASTOrderByElement.h b/dbms/include/DB/Parsers/ASTOrderByElement.h index f341265d93b..d7103a48e64 100644 --- a/dbms/include/DB/Parsers/ASTOrderByElement.h +++ b/dbms/include/DB/Parsers/ASTOrderByElement.h @@ -29,6 +29,18 @@ public: String getID() const override { return "OrderByElement"; } ASTPtr clone() const override { return new ASTOrderByElement(*this); } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + children.front()->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") << (direction == -1 ? " DESC" : " ASC") << (settings.hilite ? hilite_none : ""); + if (!collator.isNull()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "") + << "'" << collator->getLocale() << "'"; + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTQueryWithOutput.h b/dbms/include/DB/Parsers/ASTQueryWithOutput.h index 92cba621eab..3a0bb4eac0f 100644 --- a/dbms/include/DB/Parsers/ASTQueryWithOutput.h +++ b/dbms/include/DB/Parsers/ASTQueryWithOutput.h @@ -24,7 +24,7 @@ public: /// Объявляет класс-наследник ASTQueryWithOutput с реализованными методами getID и clone. -#define DEFINE_AST_QUERY_WITH_OUTPUT(Name, ID) \ +#define DEFINE_AST_QUERY_WITH_OUTPUT(Name, ID, Query) \ class Name : public ASTQueryWithOutput \ { \ public: \ @@ -44,6 +44,12 @@ public: \ } \ return ptr; \ } \ +\ +protected: \ + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \ + { \ + settings.ostr << (settings.hilite ? hilite_keyword : "") << Query << (settings.hilite ? hilite_none : ""); \ + } \ }; } diff --git a/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h b/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h index 32ebb1e528d..d99025593d0 100644 --- a/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h +++ b/dbms/include/DB/Parsers/ASTQueryWithTableAndOutput.h @@ -6,26 +6,40 @@ namespace DB { - - - /** Запрос с указанием названия таблицы и, возможно, БД и секцией FORMAT. - */ - class ASTQueryWithTableAndOutput : public ASTQueryWithOutput + + +/** Запрос с указанием названия таблицы и, возможно, БД и секцией FORMAT. + */ +class ASTQueryWithTableAndOutput : public ASTQueryWithOutput +{ +public: + String database; + String table; + + ASTQueryWithTableAndOutput() = default; + ASTQueryWithTableAndOutput(const StringRange range_) : ASTQueryWithOutput(range_) {} + +protected: + void formatHelper(const FormatSettings & settings, FormatState & state, FormatStateStacked frame, const char * name) const { - public: - String database; - String table; - - ASTQueryWithTableAndOutput() = default; - ASTQueryWithTableAndOutput(const StringRange range_) : ASTQueryWithOutput(range_) {} - }; - - + settings.ostr << (settings.hilite ? hilite_keyword : "") << name << " " << (settings.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + + if (format) + { + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << indent_str << "FORMAT " << (settings.hilite ? hilite_none : ""); + format->formatImpl(settings, state, frame); + } + } +}; + + /// Объявляет класс-наследник ASTQueryWithTableAndOutput с реализованными методами getID и clone. -#define DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(Name, ID) \ +#define DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(Name, ID, Query) \ class Name : public ASTQueryWithTableAndOutput \ { \ -public: \ + public: \ Name() = default; \ Name(const StringRange range_) : ASTQueryWithTableAndOutput(range_) {} \ String getID() const override { return ID"_" + database + "_" + table; }; \ @@ -42,5 +56,11 @@ public: \ } \ return ptr; \ } \ + \ + protected: \ + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \ + { \ + formatHelper(settings, state, frame, Query); \ + } \ }; } diff --git a/dbms/include/DB/Parsers/ASTRenameQuery.h b/dbms/include/DB/Parsers/ASTRenameQuery.h index 4eb6624e4c3..ffb59c3f0f8 100644 --- a/dbms/include/DB/Parsers/ASTRenameQuery.h +++ b/dbms/include/DB/Parsers/ASTRenameQuery.h @@ -34,6 +34,22 @@ public: String getID() const override { return "Rename"; }; ASTPtr clone() const override { return new ASTRenameQuery(*this); } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME TABLE " << (settings.hilite ? hilite_none : ""); + + for (ASTRenameQuery::Elements::const_iterator it = elements.begin(); it != elements.end(); ++it) + { + if (it != elements.begin()) + settings.ostr << ", "; + + settings.ostr << (!it->from.database.empty() ? backQuoteIfNeed(it->from.database) + "." : "") << backQuoteIfNeed(it->from.table) + << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "") + << (!it->to.database.empty() ? backQuoteIfNeed(it->to.database) + "." : "") << backQuoteIfNeed(it->to.table); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTSelectQuery.h b/dbms/include/DB/Parsers/ASTSelectQuery.h index f6edf7ebfb9..6839f6616c9 100644 --- a/dbms/include/DB/Parsers/ASTSelectQuery.h +++ b/dbms/include/DB/Parsers/ASTSelectQuery.h @@ -50,6 +50,7 @@ public: ASTPtr select_expression_list; ASTPtr database; ASTPtr table; /// Идентификатор, табличная функция или подзапрос (рекурсивно ASTSelectQuery) + bool array_join_is_left = false; /// LEFT ARRAY JOIN ASTPtr array_join_expression_list; /// ARRAY JOIN ASTPtr join; /// Обычный (не ARRAY) JOIN. bool final = false; @@ -67,6 +68,9 @@ public: ASTPtr prev_union_all; /// Следующий запрос SELECT в цепочке UNION ALL, если такой есть ASTPtr next_union_all; + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; } diff --git a/dbms/include/DB/Parsers/ASTSet.h b/dbms/include/DB/Parsers/ASTSet.h index 18edddc999f..d3af0b5f30f 100644 --- a/dbms/include/DB/Parsers/ASTSet.h +++ b/dbms/include/DB/Parsers/ASTSet.h @@ -22,6 +22,17 @@ public: String getID() const override { return "Set_" + getColumnName(); } ASTPtr clone() const override { return new ASTSet(*this); } String getColumnName() const override { return column_name; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + /** Подготовленное множество. В пользовательских запросах такого не бывает, но такое бывает после промежуточных преобразований запроса. + * Выведем его не по-настоящему (это не будет корректным запросом, но покажет, что здесь было множество). + */ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "(...)" + << (settings.hilite ? hilite_none : ""); + } }; } diff --git a/dbms/include/DB/Parsers/ASTSetQuery.h b/dbms/include/DB/Parsers/ASTSetQuery.h index 0334d1167bb..471c76d2855 100644 --- a/dbms/include/DB/Parsers/ASTSetQuery.h +++ b/dbms/include/DB/Parsers/ASTSetQuery.h @@ -26,11 +26,25 @@ public: ASTSetQuery() = default; ASTSetQuery(const StringRange range_) : IAST(range_) {} - + /** Получить текст, который идентифицирует этот элемент. */ String getID() const override { return "Set"; }; ASTPtr clone() const override { return new ASTSetQuery(*this); } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SET " << (global ? "GLOBAL " : "") << (settings.hilite ? hilite_none : ""); + + for (ASTSetQuery::Changes::const_iterator it = changes.begin(); it != changes.end(); ++it) + { + if (it != changes.begin()) + settings.ostr << ", "; + + settings.ostr << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTShowProcesslistQuery.h b/dbms/include/DB/Parsers/ASTShowProcesslistQuery.h index 8d06950319e..c1bd4f35eb3 100644 --- a/dbms/include/DB/Parsers/ASTShowProcesslistQuery.h +++ b/dbms/include/DB/Parsers/ASTShowProcesslistQuery.h @@ -5,5 +5,7 @@ namespace DB { - DEFINE_AST_QUERY_WITH_OUTPUT(ASTShowProcesslistQuery, "ShowProcesslistQuery") + +DEFINE_AST_QUERY_WITH_OUTPUT(ASTShowProcesslistQuery, "ShowProcesslistQuery", "SHOW PROCESSLIST") + } diff --git a/dbms/include/DB/Parsers/ASTShowTablesQuery.h b/dbms/include/DB/Parsers/ASTShowTablesQuery.h index 4b51b8f1aba..ccd40bb164e 100644 --- a/dbms/include/DB/Parsers/ASTShowTablesQuery.h +++ b/dbms/include/DB/Parsers/ASTShowTablesQuery.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -20,7 +21,7 @@ public: ASTShowTablesQuery() = default; ASTShowTablesQuery(const StringRange range_) : ASTQueryWithOutput(range_) {} - + /** Получить текст, который идентифицирует этот элемент. */ String getID() const override { return "ShowTables"; }; @@ -30,15 +31,43 @@ public: ASTPtr ptr{res}; res->children.clear(); - + if (format) { res->format = format->clone(); res->children.push_back(res->format); } - + return ptr; } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + if (databases) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW DATABASES" << (settings.hilite ? hilite_none : ""); + } + else + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW TABLES" << (settings.hilite ? hilite_none : ""); + + if (!from.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(from); + + if (!like.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "") + << mysqlxx::quote << like; + } + + if (format) + { + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << indent_str << "FORMAT " << (settings.hilite ? hilite_none : ""); + format->formatImpl(settings, state, frame); + } + } }; } diff --git a/dbms/include/DB/Parsers/ASTSubquery.h b/dbms/include/DB/Parsers/ASTSubquery.h index f94cbfe8346..55d7621129f 100644 --- a/dbms/include/DB/Parsers/ASTSubquery.h +++ b/dbms/include/DB/Parsers/ASTSubquery.h @@ -2,7 +2,7 @@ #include -#include +#include namespace DB @@ -11,12 +11,12 @@ namespace DB /** Подзарос SELECT */ -class ASTSubquery : public IAST +class ASTSubquery : public ASTWithAlias { public: ASTSubquery() = default; - ASTSubquery(const StringRange range_) : IAST(range_) {} - + ASTSubquery(const StringRange range_) : ASTWithAlias(range_) {} + /** Получить текст, который идентифицирует этот элемент. */ String getID() const override { return "Subquery"; } @@ -34,6 +34,20 @@ public: } String getColumnName() const override { return getTreeID(); } + +protected: + void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); + std::string nl_or_nothing = settings.one_line ? "" : "\n"; + + settings.ostr << nl_or_nothing << indent_str << "(" << nl_or_nothing; + FormatStateStacked frame_nested = frame; + frame_nested.need_parens = false; + ++frame_nested.indent; + children[0]->formatImpl(settings, state, frame_nested); + settings.ostr << nl_or_nothing << indent_str << ")"; + } }; } diff --git a/dbms/include/DB/Parsers/ASTUseQuery.h b/dbms/include/DB/Parsers/ASTUseQuery.h index eafe3496293..a1e354b39b0 100644 --- a/dbms/include/DB/Parsers/ASTUseQuery.h +++ b/dbms/include/DB/Parsers/ASTUseQuery.h @@ -16,11 +16,18 @@ public: ASTUseQuery() = default; ASTUseQuery(const StringRange range_) : IAST(range_) {} - + /** Получить текст, который идентифицирует этот элемент. */ String getID() const override { return "UseQuery_" + database; }; ASTPtr clone() const override { return new ASTUseQuery(*this); } + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database); + return; + } }; } diff --git a/dbms/include/DB/Parsers/ASTWithAlias.h b/dbms/include/DB/Parsers/ASTWithAlias.h index 63eaa186cd3..53a888baf32 100644 --- a/dbms/include/DB/Parsers/ASTWithAlias.h +++ b/dbms/include/DB/Parsers/ASTWithAlias.h @@ -1,11 +1,13 @@ #pragma once +#include #include namespace DB { + /** Базовый класс для AST, которые могут содержать алиас (идентификаторы, литералы, функции). */ class ASTWithAlias : public IAST @@ -19,10 +21,16 @@ public: String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; } String tryGetAlias() const override { return alias; } void setAlias(const String & to) override { alias = to; } + + /// Вызывает formatImplWithoutAlias, а также выводит алиас. Если надо - заключает всё выражение в скобки. + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override final; + + virtual void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const = 0; }; /// helper for setting aliases and chaining result to other functions -inline ASTPtr setAlias(ASTPtr ast, const String & alias) { +inline ASTPtr setAlias(ASTPtr ast, const String & alias) +{ ast->setAlias(alias); return ast; }; diff --git a/dbms/include/DB/Parsers/IAST.h b/dbms/include/DB/Parsers/IAST.h index 803cc9eb31f..e92b0877c7a 100644 --- a/dbms/include/DB/Parsers/IAST.h +++ b/dbms/include/DB/Parsers/IAST.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include @@ -14,8 +16,6 @@ #include #include -#include - namespace DB { @@ -133,6 +133,68 @@ public: (*it)->collectIdentifierNames(set); } + + /// Преобразовать в строку. + + /// Настройки формата. + struct FormatSettings + { + std::ostream & ostr; + bool hilite; + bool one_line; + + char nl_or_ws; + + FormatSettings(std::ostream & ostr_, bool hilite_, bool one_line_) + : ostr(ostr_), hilite(hilite_), one_line(one_line_) + { + nl_or_ws = one_line ? ' ' : '\n'; + } + }; + + /// Состояние. Например, может запоминаться множество узлов, которых мы уже обошли. + struct FormatState + { + /** Запрос SELECT, в котором найден алиас; идентификатор узла с таким алиасом. + * Нужно, чтобы когда узел встретился повторно, выводить только алиас. + */ + std::set> printed_asts_with_alias; + }; + + /// Состояние, которое копируется при форматировании каждого узла. Например, уровень вложенности. + struct FormatStateStacked + { + UInt8 indent = 0; + bool need_parens = false; + const IAST * current_select = nullptr; + }; + + void format(const FormatSettings & settings) const + { + FormatState state; + formatImpl(settings, state, FormatStateStacked()); + } + + virtual void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const + { + throw Exception("Unknown element in AST: " + getID() + + ((range.first && (range.second > range.first)) + ? " '" + std::string(range.first, range.second - range.first) + "'" + : ""), + ErrorCodes::UNKNOWN_ELEMENT_IN_AST); + } + + void writeAlias(const String & name, std::ostream & s, bool hilite) const; + +protected: + /// Для подсветки синтаксиса. + static const char * hilite_keyword; + static const char * hilite_identifier; + static const char * hilite_function; + static const char * hilite_operator; + static const char * hilite_alias; + static const char * hilite_none; + private: size_t checkDepthImpl(size_t max_depth, size_t level) const { @@ -152,4 +214,9 @@ private: typedef SharedPtr ASTPtr; typedef std::vector ASTs; + +/// Квотировать идентификатор обратными кавычками, если это требуется. +String backQuoteIfNeed(const String & x); + + } diff --git a/dbms/include/DB/Parsers/TablePropertiesQueriesASTs.h b/dbms/include/DB/Parsers/TablePropertiesQueriesASTs.h index eb06c893167..793ee655b7d 100644 --- a/dbms/include/DB/Parsers/TablePropertiesQueriesASTs.h +++ b/dbms/include/DB/Parsers/TablePropertiesQueriesASTs.h @@ -5,17 +5,9 @@ namespace DB { - - /** EXISTS запрос - */ - DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTExistsQuery, "ExistsQuery") - - /** SHOW CREATE TABLE запрос - */ - DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTShowCreateQuery, "ShowCreateQuery") - - /** DESCRIBE TABLE запрос - */ - DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTDescribeQuery, "DescribeQuery") - + +DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTExistsQuery, "ExistsQuery", "EXISTS TABLE") +DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTShowCreateQuery, "ShowCreateQuery", "SHOW CREATE TABLE") +DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTDescribeQuery, "DescribeQuery", "DESCRIBE TABLE") + } diff --git a/dbms/include/DB/Parsers/formatAST.h b/dbms/include/DB/Parsers/formatAST.h index 811d946f044..d94602e9d66 100644 --- a/dbms/include/DB/Parsers/formatAST.h +++ b/dbms/include/DB/Parsers/formatAST.h @@ -12,11 +12,14 @@ namespace DB /** Берёт синтаксическое дерево и превращает его обратно в текст. * В случае запроса INSERT, данные будут отсутствовать. */ -void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +inline void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false) +{ + IAST::FormatSettings settings(s, hilite, one_line); + ast.format(settings); +} String formatColumnsForCreateQuery(NamesAndTypesList & columns); -String backQuoteIfNeed(const String & x); inline std::ostream & operator<<(std::ostream & os, const IAST & ast) { return formatAST(ast, os, 0, false, true), os; } inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast) { return formatAST(*ast, os, 0, false, true), os; } diff --git a/dbms/include/DB/Storages/MarkCache.h b/dbms/include/DB/Storages/MarkCache.h index 2bea8630824..d556f9b204e 100644 --- a/dbms/include/DB/Storages/MarkCache.h +++ b/dbms/include/DB/Storages/MarkCache.h @@ -6,34 +6,13 @@ #include #include #include +#include + namespace DB { -struct MarkInCompressedFile -{ - size_t offset_in_compressed_file; - size_t offset_in_decompressed_block; - - bool operator==(const MarkInCompressedFile & rhs) const - { - return std::forward_as_tuple(offset_in_compressed_file, offset_in_decompressed_block) == - std::forward_as_tuple(rhs.offset_in_compressed_file, rhs.offset_in_decompressed_block); - } - bool operator!=(const MarkInCompressedFile & rhs) const - { - return !(*this == rhs); - } - - String toString() const - { - return "(" + DB::toString(offset_in_compressed_file) + "," + DB::toString(offset_in_decompressed_block) + ")"; - } -}; - -typedef std::vector MarksInCompressedFile; - /// Оценка количества байтов, занимаемых засечками в кеше. struct MarksWeightFunction { diff --git a/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h b/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h index 3cfdc8d8942..d80556903d2 100644 --- a/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h +++ b/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h @@ -23,19 +23,16 @@ public: { DayNum_t left_date; DayNum_t right_date; - UInt64 left; - UInt64 right; + Int64 left; + Int64 right; UInt32 level; std::string name; - DayNum_t left_month; - DayNum_t right_month; + DayNum_t month; bool operator<(const Part & rhs) const { - if (left_month != rhs.left_month) - return left_month < rhs.left_month; - if (right_month != rhs.right_month) - return right_month < rhs.right_month; + if (month != rhs.month) + return month < rhs.month; if (left != rhs.left) return left < rhs.left; @@ -48,8 +45,7 @@ public: /// Содержит другой кусок (получен после объединения другого куска с каким-то ещё) bool contains(const Part & rhs) const { - return left_month == rhs.left_month /// Куски за разные месяцы не объединяются - && right_month == rhs.right_month + return month == rhs.month /// Куски за разные месяцы не объединяются && left_date <= rhs.left_date && right_date >= rhs.right_date && left <= rhs.left @@ -66,7 +62,7 @@ public: size_t size() const; - static String getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level); + static String getPartName(DayNum_t left_date, DayNum_t right_date, Int64 left_id, Int64 right_id, UInt64 level); /// Возвращает true если имя директории совпадает с форматом имени директории кусочков static bool isPartDirectory(const String & dir_name, Poco::RegularExpression::MatchVec * out_matches = nullptr); diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h index 80d3788eafc..d7a163d64ca 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -33,7 +33,7 @@ public: auto part_blocks = storage.writer.splitBlockIntoParts(block); for (auto & current_block : part_blocks) { - UInt64 temp_index = storage.increment.get(); + Int64 temp_index = storage.increment.get(); MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, temp_index); storage.data.renameTempPartAndAdd(part, &storage.increment); diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h index 5c814ecb696..06d7f101b80 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h @@ -625,7 +625,7 @@ public: || mode == Mode::Aggregating; } - UInt64 getMaxDataPartIndex(); + Int64 getMaxDataPartIndex(); std::string getTableName() const override { @@ -779,6 +779,13 @@ public: return it == std::end(column_sizes) ? 0 : it->second; } + using ColumnSizes = std::unordered_map; + ColumnSizes getColumnSizes() const + { + Poco::ScopedLock lock{data_parts_mutex}; + return column_sizes; + } + /// Для ATTACH/DETACH/DROP PARTITION. static String getMonthName(const Field & partition); static DayNum_t getMonthDayNum(const Field & partition); @@ -810,7 +817,7 @@ private: NamesAndTypesListPtr columns; /// Актуальные размеры столбцов в сжатом виде - std::unordered_map column_sizes; + ColumnSizes column_sizes; BrokenPartCallback broken_part_callback; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h b/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h index 91314c92edb..c520e0349d5 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h @@ -43,7 +43,7 @@ public: * temp_index - значение left и right для нового куска. Можно будет изменить при переименовании. * Возвращает кусок с именем, начинающимся с tmp_, еще не добавленный в MergeTreeData. */ - MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithDateInterval & block, UInt64 temp_index); + MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithDateInterval & block, Int64 temp_index); private: MergeTreeData & data; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index e52d2d6a725..4e76ba89e89 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -247,7 +247,7 @@ private: ++right; } - /// Если правее засечек нет, просто используем DEFAULT_BUFFER_SIZE + /// Если правее засечек нет, просто используем max_read_buffer_size if (right >= (*marks).size() || (right + 1 == (*marks).size() && (*marks)[right].offset_in_compressed_file == (*marks)[all_mark_ranges[i].end].offset_in_compressed_file)) { diff --git a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h index aebdaee0143..80e460c30ed 100644 --- a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h @@ -14,6 +14,8 @@ namespace DB { + + class IMergedBlockOutputStream : public IBlockOutputStream { public: @@ -230,7 +232,9 @@ protected: CompressionMethod compression_method; }; -/** Для записи одного куска. Данные уже отсортированы, относятся к одному месяцу, и пишутся в один кускок. + +/** Для записи одного куска. + * Данные относятся к одному месяцу, и пишутся в один кускок. */ class MergedBlockOutputStream : public IMergedBlockOutputStream { @@ -278,45 +282,18 @@ public: } } + /// Если данные заранее отсортированы. void write(const Block & block) override { - size_t rows = block.rows(); + writeImpl(block, nullptr); + } - /// Сначала пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки. - typedef std::vector PrimaryColumns; - PrimaryColumns primary_columns; - - for (const auto & descr : storage.getSortDescription()) - primary_columns.push_back( - !descr.column_name.empty() - ? &block.getByName(descr.column_name) - : &block.getByPosition(descr.column_number)); - - for (size_t i = index_offset; i < rows; i += storage.index_granularity) - { - for (PrimaryColumns::const_iterator it = primary_columns.begin(); it != primary_columns.end(); ++it) - { - if (storage.mode != MergeTreeData::Unsorted) - index_vec.push_back((*(*it)->column)[i]); - - (*it)->type->serializeBinary(index_vec.back(), *index_stream); - } - - ++marks_count; - } - - /// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз - OffsetColumns offset_columns; - - /// Теперь пишем данные. - for (const auto & it : columns_list) - { - const ColumnWithTypeAndName & column = block.getByName(it.name); - writeData(column.name, *column.type, *column.column, offset_columns); - } - - size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity; - index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity; + /** Если данные не отсортированы, но мы заранее вычислили перестановку, после которой они станут сортированными. + * Этот метод используется для экономии оперативки, так как не нужно держать одновременно два блока - исходный и отсортированный. + */ + void writeWithPermutation(const Block & block, const IColumn::Permutation * permutation) + { + writeImpl(block, permutation); } void writeSuffix() override @@ -391,6 +368,86 @@ private: } } + /** Если задана permutation, то переставляет значения в столбцах при записи. + * Это нужно, чтобы не держать целый блок в оперативке для его сортировки. + */ + void writeImpl(const Block & block, const IColumn::Permutation * permutation) + { + size_t rows = block.rows(); + + /// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз + OffsetColumns offset_columns; + + auto sort_description = storage.getSortDescription(); + + /// Сюда будем складывать столбцы, относящиеся к Primary Key, чтобы потом записать индекс. + std::vector primary_columns(sort_description.size()); + std::map primary_columns_name_to_position; + + for (size_t i = 0, size = sort_description.size(); i < size; ++i) + { + const auto & descr = sort_description[i]; + + String name = !descr.column_name.empty() + ? descr.column_name + : block.getByPosition(descr.column_number).name; + + if (!primary_columns_name_to_position.emplace(name, i).second) + throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS); + + primary_columns[i] = !descr.column_name.empty() + ? block.getByName(descr.column_name) + : block.getByPosition(descr.column_number); + + /// Столбцы первичного ключа переупорядочиваем заранее и складываем в primary_columns. + if (permutation) + primary_columns[i].column = primary_columns[i].column->permute(*permutation, 0); + } + + /// Теперь пишем данные. + for (const auto & it : columns_list) + { + const ColumnWithTypeAndName & column = block.getByName(it.name); + + if (permutation) + { + auto primary_column_it = primary_columns_name_to_position.find(it.name); + if (primary_columns_name_to_position.end() != primary_column_it) + { + writeData(column.name, *column.type, *primary_columns[primary_column_it->second].column, offset_columns); + } + else + { + /// Столбцы, не входящие в первичный ключ, переупорядочиваем здесь; затем результат освобождается - для экономии оперативки. + ColumnPtr permutted_column = column.column->permute(*permutation, 0); + writeData(column.name, *column.type, *permutted_column, offset_columns); + } + } + else + { + writeData(column.name, *column.type, *column.column, offset_columns); + } + } + + /// Пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки. + for (size_t i = index_offset; i < rows; i += storage.index_granularity) + { + if (storage.mode != MergeTreeData::Unsorted) + { + for (const auto & primary_column : primary_columns) + { + index_vec.push_back((*primary_column.column)[i]); + primary_column.type->serializeBinary(index_vec.back(), *index_stream); + } + } + + ++marks_count; + } + + size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity; + index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity; + } + private: NamesAndTypesList columns_list; String part_path; diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 72aee9e0e62..cbac9398b36 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -38,7 +38,7 @@ public: AbandonableLockInZooKeeper block_number_lock = storage.allocateBlockNumber(month_name); - UInt64 part_number = block_number_lock.getNumber(); + Int64 part_number = block_number_lock.getNumber(); MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, part_number); String part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level); diff --git a/dbms/include/DB/Storages/StorageLog.h b/dbms/include/DB/Storages/StorageLog.h index 626b04e6929..5acc06fda80 100644 --- a/dbms/include/DB/Storages/StorageLog.h +++ b/dbms/include/DB/Storages/StorageLog.h @@ -84,8 +84,6 @@ public: }; typedef std::map Files_t; - Files_t & getFiles() { return files; } - bool checkData() const override; protected: @@ -149,7 +147,7 @@ private: size_t max_compress_block_size; protected: - FileChecker file_checker; + FileChecker file_checker; private: /** Для обычных столбцов, в засечках указано количество строчек в блоке. diff --git a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h index db85fdede2c..225945fa3a3 100644 --- a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h +++ b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h @@ -393,8 +393,11 @@ private: */ void waitForReplicaToProcessLogEntry(const String & replica_name, const LogEntry & entry); - /// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper. - static String padIndex(UInt64 index) + /** Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper. + * Поддерживаются также отрицательные числа - для них имя ноды выглядит несколько глупо + * и не соответствует никакой автоинкрементной ноде в ZK. + */ + static String padIndex(Int64 index) { String index_str = toString(index); return std::string(10 - index_str.size(), '0') + index_str; diff --git a/dbms/include/DB/Storages/StorageStripeLog.h b/dbms/include/DB/Storages/StorageStripeLog.h new file mode 100644 index 00000000000..d12642e0963 --- /dev/null +++ b/dbms/include/DB/Storages/StorageStripeLog.h @@ -0,0 +1,89 @@ +#pragma once + +#include +#include + +#include +#include + + +namespace DB +{ + +/** Реализует хранилище, подходящее для маленьких кусочков лога. + * При этом, хранит все столбцы в одном файле формата Native, с расположенным рядом индексом. + */ +class StorageStripeLog : public IStorage +{ +friend class StripeLogBlockInputStream; +friend class StripeLogBlockOutputStream; + +public: + /** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце), + * (корректность имён и путей не проверяется) + * состоящую из указанных столбцов. + * Если не указано attach - создать директорию, если её нет. + */ + static StoragePtr create( + const std::string & path_, + const std::string & name_, + NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + bool attach, + size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE); + + std::string getName() const override { return "StripeLog"; } + std::string getTableName() const override { return name; } + + const NamesAndTypesList & getColumnsListImpl() const override { return *columns; } + + BlockInputStreams read( + const Names & column_names, + ASTPtr query, + const Context & context, + const Settings & settings, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size = DEFAULT_BLOCK_SIZE, + unsigned threads = 1) override; + + BlockOutputStreamPtr write(ASTPtr query) override; + + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + + bool checkData() const override; + + /// Данные файла. + struct ColumnData + { + Poco::File data_file; + }; + typedef std::map Files_t; + + std::string full_path() { return path + escapeForFileName(name) + '/';} + +private: + String path; + String name; + NamesAndTypesListPtr columns; + + size_t max_compress_block_size; + + FileChecker file_checker; + Poco::RWLock rwlock; + + Logger * log; + + StorageStripeLog( + const std::string & path_, + const std::string & name_, + NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + bool attach, + size_t max_compress_block_size_); +}; + +} diff --git a/dbms/include/DB/Storages/StorageTinyLog.h b/dbms/include/DB/Storages/StorageTinyLog.h index e0c4ecab489..2a0c31c47a2 100644 --- a/dbms/include/DB/Storages/StorageTinyLog.h +++ b/dbms/include/DB/Storages/StorageTinyLog.h @@ -64,8 +64,6 @@ public: }; typedef std::map Files_t; - Files_t & getFiles(); - std::string full_path() { return path + escapeForFileName(name) + '/';} private: @@ -77,7 +75,7 @@ private: Files_t files; - FileChecker file_checker; + FileChecker file_checker; Logger * log; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 4676d21bdda..245530b86d6 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -652,7 +652,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("State")), argument_types, recursion_level + 1); return new AggregateFunctionState(nested); } - else if (recursion_level == 0 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge"))) + else if (recursion_level <= 1 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge"))) { /// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции. if (argument_types.size() != 1) @@ -668,7 +668,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da return new AggregateFunctionMerge(nested); } - else if (recursion_level <= 1 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f') + else if (recursion_level <= 2 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f') { if (argument_types.empty()) throw Exception{ @@ -682,7 +682,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da AggregateFunctionPtr nested = get(String(name.data(), name.size() - 2), nested_dt, recursion_level + 1); return new AggregateFunctionIf(nested); } - else if (recursion_level <= 2 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array"))) + else if (recursion_level <= 3 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array"))) { /// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции. size_t num_agruments = argument_types.size(); @@ -695,7 +695,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da else throw Exception("Illegal type " + argument_types[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + name + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("Array")), nested_arguments, recursion_level + 2); /// + 2, чтобы ни один другой модификатор не мог идти перед Array + AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("Array")), nested_arguments, recursion_level + 3); /// + 3, чтобы ни один другой модификатор не мог идти перед Array return new AggregateFunctionArray(nested); } else @@ -765,14 +765,14 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int if (recursion_level <= 0 && name.size() > strlen("State") && !(strcmp(name.data() + name.size() - strlen("State"), "State"))) return isAggregateFunctionName(String(name.data(), name.size() - strlen("State")), recursion_level + 1); /// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции. - if (recursion_level <= 0 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge"))) + if (recursion_level <= 1 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge"))) return isAggregateFunctionName(String(name.data(), name.size() - strlen("Merge")), recursion_level + 1); /// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции. - if (recursion_level <= 1 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f') + if (recursion_level <= 2 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f') return isAggregateFunctionName(String(name.data(), name.size() - 2), recursion_level + 1); /// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции. - if (recursion_level <= 2 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array"))) - return isAggregateFunctionName(String(name.data(), name.size() - strlen("Array")), recursion_level + 2); /// + 2, чтобы ни один другой модификатор не мог идти перед Array + if (recursion_level <= 3 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array"))) + return isAggregateFunctionName(String(name.data(), name.size() - strlen("Array")), recursion_level + 3); /// + 3, чтобы ни один другой модификатор не мог идти перед Array return false; } diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index a92ab6c4935..4c5a90b3796 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -369,6 +369,19 @@ private: } + /** Проверка для случая, когда в терминал вставляется многострочный запрос из буфера обмена. + * Позволяет не начинать выполнение одной строчки запроса, пока весь запрос не будет вставлен. + */ + static bool hasDataInSTDIN() + { + timeval timeout = { 0, 0 }; + fd_set fds; + FD_ZERO(&fds); + FD_SET(STDIN_FILENO, &fds); + return select(1, &fds, 0, 0, &timeout) == 1; + } + + void loop() { String query; @@ -395,7 +408,7 @@ private: query += line; - if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || !config().has("multiline"))) + if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || (!config().has("multiline") && !hasDataInSTDIN()))) { if (query != prev_query) { @@ -464,6 +477,12 @@ private: copyData(in, out); } + process(line); + } + + + bool process(const String & line) + { if (config().has("multiquery")) { /// Несколько запросов, разделенных ';'. @@ -494,17 +513,20 @@ private: while (isWhitespace(*begin) || *begin == ';') ++begin; - process(query, ast); + if (!processSingleQuery(query, ast)) + return false; } + + return true; } else { - process(line); + return processSingleQuery(line); } } - bool process(const String & line, ASTPtr parsed_query_ = nullptr) + bool processSingleQuery(const String & line, ASTPtr parsed_query_ = nullptr) { if (exit_strings.end() != exit_strings.find(line)) return false; @@ -838,15 +860,8 @@ private: } - void onData(Block & block) + void initBlockOutputStream(const Block & block) { - if (written_progress_chars) - clearProgress(); - - if (!block) - return; - - processed_rows += block.rows(); if (!block_std_out) { String current_format = format; @@ -869,8 +884,21 @@ private: block_std_out = context.getFormatFactory().getOutput(current_format, std_out, block); block_std_out->writePrefix(); } + } - /// Загаловочный блок с нулем строк использовался для инициализации block_std_out, + + void onData(Block & block) + { + if (written_progress_chars) + clearProgress(); + + if (!block) + return; + + processed_rows += block.rows(); + initBlockOutputStream(block); + + /// Заголовочный блок с нулем строк использовался для инициализации block_std_out, /// выводить его не нужно if (block.rows() != 0) { @@ -885,11 +913,13 @@ private: void onTotals(Block & block) { + initBlockOutputStream(block); block_std_out->setTotals(block); } void onExtremes(Block & block) { + initBlockOutputStream(block); block_std_out->setExtremes(block); } diff --git a/dbms/src/Client/ParallelReplicas.cpp b/dbms/src/Client/ParallelReplicas.cpp index 135c1b06aa0..0702482f4c9 100644 --- a/dbms/src/Client/ParallelReplicas.cpp +++ b/dbms/src/Client/ParallelReplicas.cpp @@ -214,7 +214,7 @@ Connection::Packet ParallelReplicas::receivePacketUnlocked() auto it = getReplicaForReading(); if (it == replica_map.end()) - throw Exception("No available replica", ErrorCodes::NO_AVAILABLE_REPLICA); + throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA); Connection * connection = it->second; Connection::Packet packet = connection->receivePacket(); @@ -263,9 +263,8 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent() Poco::Net::Socket::SocketList read_list; read_list.reserve(active_replica_count); - /** Сначала проверяем, есть ли данные, которые уже лежат в буфере - * хоть одного соединения. - */ + /// Сначала проверяем, есть ли данные, которые уже лежат в буфере + /// хоть одного соединения. for (auto & e : replica_map) { Connection * connection = e.second; @@ -273,9 +272,8 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent() read_list.push_back(connection->socket); } - /** Если не было найдено никаких данных, то проверяем, есть ли соединения - * готовые для чтения. - */ + /// Если не было найдено никаких данных, то проверяем, есть ли соединения + /// готовые для чтения. if (read_list.empty()) { Poco::Net::Socket::SocketList write_list; @@ -287,9 +285,17 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent() if (connection != nullptr) read_list.push_back(connection->socket); } - int n = Poco::Net::Socket::select(read_list, write_list, except_list, settings->poll_interval * 1000000); + + int n = Poco::Net::Socket::select(read_list, write_list, except_list, settings->receive_timeout); + if (n == 0) - return replica_map.end(); + { + std::stringstream description; + for (auto it = replica_map.begin(); it != replica_map.end(); ++it) + description << (it != replica_map.begin() ? ", " : "") << it->second->getDescription(); + + throw Exception("Timeout exceeded while reading from " + description.str(), ErrorCodes::TIMEOUT_EXCEEDED); + } } auto & socket = read_list[rand() % read_list.size()]; diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 7c827846980..4d786c94a6e 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -141,9 +141,12 @@ void Block::insertUnique(const ColumnWithTypeAndName & elem) void Block::erase(size_t position) { + if (index_by_position.empty()) + throw Exception("Block is empty", ErrorCodes::POSITION_OUT_OF_BOUND); + if (position >= index_by_position.size()) throw Exception("Position out of bound in Block::erase(), max position = " - + toString(index_by_position.size()), ErrorCodes::POSITION_OUT_OF_BOUND); + + toString(index_by_position.size() - 1), ErrorCodes::POSITION_OUT_OF_BOUND); Container_t::iterator it = index_by_position[position]; index_by_name.erase(index_by_name.find(it->name)); @@ -177,6 +180,9 @@ void Block::erase(const String & name) ColumnWithTypeAndName & Block::getByPosition(size_t position) { + if (index_by_position.empty()) + throw Exception("Block is empty", ErrorCodes::POSITION_OUT_OF_BOUND); + if (position >= index_by_position.size()) throw Exception("Position " + toString(position) + " is out of bound in Block::getByPosition(), max position = " @@ -189,6 +195,9 @@ ColumnWithTypeAndName & Block::getByPosition(size_t position) const ColumnWithTypeAndName & Block::getByPosition(size_t position) const { + if (index_by_position.empty()) + throw Exception("Block is empty", ErrorCodes::POSITION_OUT_OF_BOUND); + if (position >= index_by_position.size()) throw Exception("Position " + toString(position) + " is out of bound in Block::getByPosition(), max position = " @@ -302,7 +311,13 @@ std::string Block::dumpStructure() const { if (it != data.begin()) res << ", "; - res << it->name << ' ' << it->type->getName() << ' ' << it->column->getName() << ' ' << it->column->size(); + + res << it->name << ' ' << it->type->getName(); + + if (it->column) + res << ' ' << it->column->getName() << ' ' << it->column->size(); + else + res << " nullptr"; } return res.str(); } diff --git a/dbms/src/Core/Exception.cpp b/dbms/src/Core/Exception.cpp index 789af7beeae..6c2c52baa72 100644 --- a/dbms/src/Core/Exception.cpp +++ b/dbms/src/Core/Exception.cpp @@ -52,16 +52,16 @@ inline std::string demangle(const char * const mangled, int & status) return demangled; } -void tryLogCurrentException(const char * log_name) +void tryLogCurrentException(const char * log_name, const std::string & start_of_message) { - tryLogCurrentException(&Logger::get(log_name)); + tryLogCurrentException(&Logger::get(log_name), start_of_message); } -void tryLogCurrentException(Poco::Logger * logger) +void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message) { try { - LOG_ERROR(logger, getCurrentExceptionMessage(true)); + LOG_ERROR(logger, start_of_message << (start_of_message.empty() ? "" : ": ") << getCurrentExceptionMessage(true)); } catch (...) { diff --git a/dbms/src/DataStreams/FormatFactory.cpp b/dbms/src/DataStreams/FormatFactory.cpp index ea82279d977..82d0b8dc913 100644 --- a/dbms/src/DataStreams/FormatFactory.cpp +++ b/dbms/src/DataStreams/FormatFactory.cpp @@ -26,7 +26,7 @@ namespace DB { BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & buf, - Block & sample, size_t max_block_size) const + const Block & sample, size_t max_block_size) const { if (name == "Native") return new NativeBlockInputStream(buf); @@ -48,7 +48,7 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf, - Block & sample) const + const Block & sample) const { if (name == "Native") return new NativeBlockOutputStream(buf); diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 54ca62c253c..0b21d86a541 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -14,6 +15,25 @@ namespace DB { +NativeBlockInputStream::NativeBlockInputStream( + ReadBuffer & istr_, UInt64 server_revision_, + bool use_index_, + IndexForNativeFormat::Blocks::const_iterator index_block_it_, + IndexForNativeFormat::Blocks::const_iterator index_block_end_) + : istr(istr_), server_revision(server_revision_), + use_index(use_index_), index_block_it(index_block_it_), index_block_end(index_block_end_) +{ + if (use_index) + { + istr_concrete = typeid_cast(&istr); + if (!istr_concrete) + throw Exception("When need to use index for NativeBlockInputStream, istr must be CompressedReadBufferFromFile.", ErrorCodes::LOGICAL_ERROR); + + index_column_it = index_block_it->columns.begin(); + } +} + + void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows) { /** Для массивов требуется сначала десериализовать смещения, а потом значения. @@ -47,9 +67,17 @@ Block NativeBlockInputStream::readImpl() const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); - if (istr.eof()) + if (use_index && index_block_it == index_block_end) return res; + if (istr.eof()) + { + if (use_index) + throw Exception("Input doesn't contain all data for index.", ErrorCodes::CANNOT_READ_ALL_DATA); + + return res; + } + /// Дополнительная информация о блоке. if (server_revision >= DBMS_MIN_REVISION_WITH_BLOCK_INFO) res.info.read(istr); @@ -57,29 +85,103 @@ Block NativeBlockInputStream::readImpl() /// Размеры size_t columns = 0; size_t rows = 0; - readVarUInt(columns, istr); - readVarUInt(rows, istr); + + if (!use_index) + { + readVarUInt(columns, istr); + readVarUInt(rows, istr); + } + else + { + columns = index_block_it->num_columns; + rows = index_block_it->num_rows; + } for (size_t i = 0; i < columns; ++i) { + if (use_index) + { + /// Если текущая позиция и так какая требуется, то реального seek-а не происходит. + istr_concrete->seek(index_column_it->location.offset_in_compressed_file, index_column_it->location.offset_in_decompressed_block); + } + ColumnWithTypeAndName column; /// Имя - readStringBinary(column.name, istr); + readBinary(column.name, istr); /// Тип String type_name; - readStringBinary(type_name, istr); + readBinary(type_name, istr); column.type = data_type_factory.get(type_name); + if (use_index) + { + /// Индекс позволяет сделать проверки. + if (index_column_it->name != column.name) + throw Exception("Index points to column with wrong name: corrupted index or data", ErrorCodes::INCORRECT_INDEX); + if (index_column_it->type != type_name) + throw Exception("Index points to column with wrong type: corrupted index or data", ErrorCodes::INCORRECT_INDEX); + } + /// Данные column.column = column.type->createColumn(); readData(*column.type, *column.column, istr, rows); res.insert(column); + + if (use_index) + ++index_column_it; + } + + if (use_index) + { + if (index_column_it != index_block_it->columns.end()) + throw Exception("Inconsistent index: not all columns were read", ErrorCodes::INCORRECT_INDEX); + + ++index_block_it; + if (index_block_it != index_block_end) + index_column_it = index_block_it->columns.begin(); } return res; } + +void IndexForNativeFormat::read(ReadBuffer & istr, const NameSet & required_columns) +{ + while (!istr.eof()) + { + blocks.emplace_back(); + IndexOfBlockForNativeFormat & block = blocks.back(); + + readVarUInt(block.num_columns, istr); + readVarUInt(block.num_rows, istr); + + if (block.num_columns < required_columns.size()) + throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX); + + for (size_t i = 0; i < block.num_columns; ++i) + { + IndexOfOneColumnForNativeFormat column_index; + + readBinary(column_index.name, istr); + readBinary(column_index.type, istr); + readBinary(column_index.location.offset_in_compressed_file, istr); + readBinary(column_index.location.offset_in_decompressed_block, istr); + + if (required_columns.count(column_index.name)) + block.columns.push_back(std::move(column_index)); + } + + if (block.columns.size() < required_columns.size()) + throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX); + if (block.columns.size() > required_columns.size()) + throw Exception("Index contain duplicate columns", ErrorCodes::INCORRECT_INDEX); + + block.num_columns = block.columns.size(); + } +} + + } diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index a85a10a5909..ccc461d6015 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -2,12 +2,14 @@ #include #include +#include #include #include #include +#include #include @@ -15,6 +17,21 @@ namespace DB { +NativeBlockOutputStream::NativeBlockOutputStream( + WriteBuffer & ostr_, UInt64 client_revision_, + WriteBuffer * index_ostr_) + : ostr(ostr_), client_revision(client_revision_), + index_ostr(index_ostr_) +{ + if (index_ostr) + { + ostr_concrete = typeid_cast(&ostr); + if (!ostr_concrete) + throw Exception("When need to write index for NativeBlockOutputStream, ostr must be CompressedWriteBuffer.", ErrorCodes::LOGICAL_ERROR); + } +} + + void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** Если есть столбцы-константы - то материализуем их. @@ -71,11 +88,31 @@ void NativeBlockOutputStream::write(const Block & block) /// Размеры size_t columns = block.columns(); size_t rows = block.rows(); + writeVarUInt(columns, ostr); writeVarUInt(rows, ostr); + /** Индекс имеет ту же структуру, что и поток с данными. + * Но вместо значений столбца он содержит засечку, ссылающуюся на место в файле с данными, где находится этот кусочек столбца. + */ + if (index_ostr) + { + writeVarUInt(columns, *index_ostr); + writeVarUInt(rows, *index_ostr); + } + for (size_t i = 0; i < columns; ++i) { + /// Для индекса. + MarkInCompressedFile mark; + + if (index_ostr) + { + ostr_concrete->next(); /// Заканчиваем сжатый блок. + mark.offset_in_compressed_file = ostr_concrete->getCompressedBytes(); + mark.offset_in_decompressed_block = ostr_concrete->getRemainingBytes(); + } + const ColumnWithTypeAndName & column = block.getByPosition(i); /// Имя @@ -86,6 +123,15 @@ void NativeBlockOutputStream::write(const Block & block) /// Данные writeData(*column.type, column.column, ostr, 0, 0); + + if (index_ostr) + { + writeStringBinary(column.name, *index_ostr); + writeStringBinary(column.type->getName(), *index_ostr); + + writeBinary(mark.offset_in_compressed_file, *index_ostr); + writeBinary(mark.offset_in_decompressed_block, *index_ostr); + } } } diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 5ae25eb0444..57b718b3ed8 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -137,12 +137,28 @@ void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr, size ColumnString::Chars_t & data = column_string.getChars(); ColumnString::Offsets_t & offsets = column_string.getOffsets(); - /// Выбрано наугад. - constexpr auto avg_value_size_hint_reserve_multiplier = 1.2; + double avg_chars_size; - double avg_chars_size = (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0]) - ? (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier - : DBMS_APPROX_STRING_SIZE); + if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0])) + { + /// Выбрано наугад. + constexpr auto avg_value_size_hint_reserve_multiplier = 1.2; + + avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier; + } + else + { + /** Небольшая эвристика для оценки того, что в столбце много пустых строк. + * В этом случае, для экономии оперативки, будем говорить, что средний размер значения маленький. + */ + if (istr.position() + sizeof(UInt32) <= istr.buffer().end() + && *reinterpret_cast(istr.position()) == 0) /// Первые 4 строки находятся в буфере и являются пустыми. + { + avg_chars_size = 1; + } + else + avg_chars_size = DBMS_APPROX_STRING_SIZE; + } data.reserve(data.size() + std::ceil(limit * avg_chars_size)); diff --git a/dbms/src/Functions/FunctionsArray.cpp b/dbms/src/Functions/FunctionsArray.cpp index d30fbff1e7f..4210e77480a 100644 --- a/dbms/src/Functions/FunctionsArray.cpp +++ b/dbms/src/Functions/FunctionsArray.cpp @@ -27,6 +27,7 @@ void registerFunctionsArray(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 74bffca822a..f21656a54de 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -1688,6 +1689,66 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants } +Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) +{ + if (blocks.empty()) + return {}; + + StringRefs key(keys_size); + ConstColumnPlainPtrs key_columns(keys_size); + + AggregateColumnsData aggregate_columns(aggregates_size); + + initialize(blocks.front()); + + /// Каким способом выполнять агрегацию? + for (size_t i = 0; i < keys_size; ++i) + key_columns[i] = sample.getByPosition(i).column; + + Sizes key_sizes; + AggregatedDataVariants::Type method = chooseAggregationMethod(key_columns, key_sizes); + + /// Временные данные для агрегации. + AggregatedDataVariants result; + + /// result будет уничтожать состояния агрегатных функций в деструкторе + result.aggregator = this; + + result.init(method); + result.keys_size = keys_size; + result.key_sizes = key_sizes; + + LOG_TRACE(log, "Merging partially aggregated blocks."); + + for (Block & block : blocks) + { + 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); + + APPLY_FOR_AGGREGATED_VARIANTS(M) + #undef M + else if (result.type != AggregatedDataVariants::Type::without_key) + throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + } + + BlocksList merged_block = convertToBlocks(result, final, 1); + + if (merged_block.size() > 1) /// TODO overflows + throw Exception("Logical error: temporary result is not single-level", ErrorCodes::LOGICAL_ERROR); + + LOG_TRACE(log, "Merged partially aggregated blocks."); + + if (merged_block.empty()) + return {}; + + return merged_block.front(); +} + + template void NO_INLINE Aggregator::destroyImpl( Method & method) const @@ -1769,4 +1830,5 @@ void Aggregator::setCancellationHook(const CancellationHook cancellation_hook) isCancelled = cancellation_hook; } + } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index ac992e66349..22024e0b851 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -96,6 +97,8 @@ struct ContextShared /// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings Poco::SharedPtr clusters; + Poco::UUIDGenerator uuid_generator; + bool shutdown_called = false; @@ -587,8 +590,12 @@ void Context::setCurrentDatabase(const String & name) void Context::setCurrentQueryId(const String & query_id) { + String query_id_to_set = query_id; + if (query_id_to_set.empty()) /// Если пользователь не передал свой query_id, то генерируем его самостоятельно. + query_id_to_set = shared->uuid_generator.createRandom().toString(); + Poco::ScopedLock lock(shared->mutex); - current_query_id = query_id; + current_query_id = query_id_to_set; } diff --git a/dbms/src/Interpreters/DictionaryFactory.cpp b/dbms/src/Interpreters/DictionaryFactory.cpp index 23a434c6eef..688fe114f20 100644 --- a/dbms/src/Interpreters/DictionaryFactory.cpp +++ b/dbms/src/Interpreters/DictionaryFactory.cpp @@ -31,6 +31,8 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + const auto & layout_type = keys.front(); if ("range_hashed" == layout_type) @@ -41,7 +43,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab ErrorCodes::BAD_ARGUMENTS }; - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); } else { @@ -49,16 +51,15 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab throw Exception{ "Elements .structure.range_min and .structure.range_max should be defined only " "for a dictionary of layout 'range_hashed'", - ErrorCodes::BAD_ARGUMENTS - }; + ErrorCodes::BAD_ARGUMENTS}; if ("flat" == layout_type) { - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); } else if ("hashed" == layout_type) { - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); } else if ("cache" == layout_type) { @@ -66,8 +67,12 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab if (size == 0) throw Exception{ "Dictionary of layout 'cache' cannot have 0 cells", - ErrorCodes::TOO_SMALL_BUFFER_SIZE - }; + ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + + if (require_nonempty) + throw Exception{ + "Dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index a227bb1c713..caa612434ce 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -268,6 +269,24 @@ void ExpressionAction::execute(Block & block) const if (!any_array) throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH); + /// Если LEFT ARRAY JOIN, то создаём столбцы, в которых пустые массивы заменены на массивы с одним элементом - значением по-умолчанию. + std::map non_empty_array_columns; + if (array_join_is_left) + { + for (const auto & name : array_joined_columns) + { + auto src_col = block.getByName(name); + + Block tmp_block{src_col, {{}, src_col.type, {}}}; + + FunctionEmptyArrayToSingle().execute(tmp_block, {0}, 1); + non_empty_array_columns[name] = tmp_block.getByPosition(1).column; + } + + any_array_ptr = non_empty_array_columns.begin()->second; + any_array = typeid_cast(&*any_array_ptr); + } + size_t columns = block.columns(); for (size_t i = 0; i < columns; ++i) { @@ -278,7 +297,8 @@ void ExpressionAction::execute(Block & block) const if (!typeid_cast(&*current.type)) throw Exception("ARRAY JOIN of not array: " + current.name, ErrorCodes::TYPE_MISMATCH); - ColumnPtr array_ptr = current.column; + ColumnPtr array_ptr = array_join_is_left ? non_empty_array_columns[current.name] : current.column; + if (array_ptr->isConst()) array_ptr = dynamic_cast(*array_ptr).convertToFullColumn(); @@ -379,7 +399,7 @@ std::string ExpressionAction::toString() const break; case ARRAY_JOIN: - ss << "ARRAY JOIN "; + ss << (array_join_is_left ? "LEFT " : "") << "ARRAY JOIN "; for (NameSet::const_iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it) { if (it != array_joined_columns.begin()) @@ -761,7 +781,7 @@ std::string ExpressionActions::getID() const ss << actions[i].result_name; if (actions[i].type == ExpressionAction::ARRAY_JOIN) { - ss << "{"; + ss << (actions[i].array_join_is_left ? "LEFT ARRAY JOIN" : "ARRAY JOIN") << "{"; for (NameSet::const_iterator it = actions[i].array_joined_columns.begin(); it != actions[i].array_joined_columns.end(); ++it) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 11787edd54a..c02ee22684a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -87,6 +87,18 @@ const std::unordered_set possibly_injective_function_names "dictGetDateTime" }; +static bool functionIsInOperator(const String & name) +{ + return name == "in" || name == "notIn"; +} + +static bool functionIsInOrGlobalInOperator(const String & name) +{ + return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn"; +} + + + void ExpressionAnalyzer::init() { select_query = typeid_cast(&*ast); @@ -95,6 +107,7 @@ void ExpressionAnalyzer::init() LogicalExpressionsOptimizer logical_expressions_optimizer(select_query, settings); logical_expressions_optimizer.optimizeDisjunctiveEqualityChains(); + /// Добавляет в множество известных алиасов те, которые объявлены в структуре таблицы (ALIAS-столбцы). addStorageAliases(); /// Создаёт словарь aliases: alias -> ASTPtr @@ -103,6 +116,9 @@ void ExpressionAnalyzer::init() /// Common subexpression elimination. Rewrite rules. normalizeTree(); + /// Выполнение скалярных подзапросов - замена их на значения-константы. + executeScalarSubqueries(); + /// GROUP BY injective function elimination. optimizeGroupBy(); @@ -223,9 +239,10 @@ void ExpressionAnalyzer::analyzeAggregation() void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() { + /// Преобразует GLOBAL-подзапросы во внешние таблицы; кладёт их в словарь external_tables: name -> StoragePtr. initGlobalSubqueries(ast); - /// Создаёт словарь external_tables: name -> StoragePtr. + /// Добавляет уже существующие внешние таблицы (не подзапросы) в словарь external_tables. findExternalTables(ast); } @@ -388,7 +405,7 @@ void ExpressionAnalyzer::normalizeTreeImpl( } /// Может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t). - if (func_node->name == "in" || func_node->name == "notIn" || func_node->name == "globalIn" || func_node->name == "globalNotIn") + if (functionIsInOrGlobalInOperator(func_node->name)) if (ASTIdentifier * right = typeid_cast(&*func_node->arguments->children.at(1))) right->kind = ASTIdentifier::Table; @@ -528,6 +545,145 @@ void ExpressionAnalyzer::normalizeTreeImpl( finished_asts[initial_ast] = ast; } + +void ExpressionAnalyzer::executeScalarSubqueries() +{ + if (!select_query) + executeScalarSubqueriesImpl(ast); + else + { + for (auto & child : ast->children) + { + /// Не опускаемся в FROM и JOIN. + if (child.get() != select_query->table.get() && child.get() != select_query->join.get()) + executeScalarSubqueriesImpl(child); + } + } +} + + +static ASTPtr addTypeConversion(ASTLiteral * ast_, const String & type_name) +{ + if (0 == type_name.compare(0, strlen("Array"), "Array")) + return ast_; /// Преобразование типов для массивов пока не поддерживаем. + + auto ast = std::unique_ptr(ast_); + ASTFunction * func = new ASTFunction(ast->range); + ASTPtr res = func; + func->alias = ast->alias; + ast->alias.clear(); + func->kind = ASTFunction::FUNCTION; + func->name = "to" + type_name; + ASTExpressionList * exp_list = new ASTExpressionList(ast->range); + func->arguments = exp_list; + func->children.push_back(func->arguments); + exp_list->children.push_back(ast.release()); + return res; +} + + +void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast) +{ + /** Заменяем подзапросы, возвращающие ровно одну строку + * ("скалярные" подзапросы) на соответствующие константы. + * + * Если подзапрос возвращает более одного столбца, то он заменяется на кортеж констант. + * + * Особенности: + * + * Замена происходит во время анализа запроса, а не во время основной стадии выполнения. + * Это значит, что не будет работать индикатор прогресса во время выполнения этих запросов, + * а также такие запросы нельзя будет прервать. + * + * Зато результат запросов может быть использован для индекса в таблице. + * + * Скалярные подзапросы выполняются на сервере-инициаторе запроса. + * На удалённые серверы запрос отправляется с уже подставленными константами. + */ + + if (ASTSubquery * subquery = typeid_cast(ast.get())) + { + Context subquery_context = context; + Settings subquery_settings = context.getSettings(); + subquery_settings.limits.max_result_rows = 1; + subquery_settings.extremes = 0; + subquery_context.setSettings(subquery_settings); + + ASTPtr query = subquery->children.at(0); + BlockIO res = InterpreterSelectQuery(query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1).execute(); + + Block block; + try + { + block = res.in->read(); + + if (!block) + throw Exception("Scalar subquery returned empty result", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + + if (block.rows() != 1 || res.in->read()) + throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::TOO_MUCH_ROWS) + throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + else + throw; + } + + size_t columns = block.columns(); + if (columns == 1) + { + ASTLiteral * lit = new ASTLiteral(ast->range, (*block.getByPosition(0).column)[0]); + lit->alias = subquery->alias; + ast = addTypeConversion(lit, block.getByPosition(0).type->getName()); + } + else + { + ASTFunction * tuple = new ASTFunction(ast->range); + tuple->alias = subquery->alias; + ast = tuple; + tuple->kind = ASTFunction::FUNCTION; + tuple->name = "tuple"; + ASTExpressionList * exp_list = new ASTExpressionList(ast->range); + tuple->arguments = exp_list; + tuple->children.push_back(tuple->arguments); + + exp_list->children.resize(columns); + for (size_t i = 0; i < columns; ++i) + { + exp_list->children[i] = addTypeConversion( + new ASTLiteral(ast->range, (*block.getByPosition(i).column)[0]), + block.getByPosition(i).type->getName()); + } + } + } + else + { + /** Не опускаемся в подзапросы в аргументах IN. + * Но если аргумент - не подзапрос, то глубже внутри него могут быть подзапросы, и в них надо опускаться. + */ + ASTFunction * func = typeid_cast(ast.get()); + if (func && func->kind == ASTFunction::FUNCTION + && functionIsInOrGlobalInOperator(func->name)) + { + for (auto & child : ast->children) + { + if (child.get() != func->arguments) + executeScalarSubqueriesImpl(child); + else + for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) + if (i != 1 || !typeid_cast(func->arguments->children[i].get())) + executeScalarSubqueriesImpl(func->arguments->children[i]); + } + } + else + for (auto & child : ast->children) + executeScalarSubqueriesImpl(child); + } +} + + void ExpressionAnalyzer::optimizeGroupBy() { if (!(select_query && select_query->group_expression_list)) @@ -657,7 +813,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl makeSetsForIndexImpl(child, sample_block); ASTFunction * func = typeid_cast(node.get()); - if (func && func->kind == ASTFunction::FUNCTION && (func->name == "in" || func->name == "notIn")) + if (func && func->kind == ASTFunction::FUNCTION && functionIsInOperator(func->name)) { IAST & args = *func->arguments; ASTPtr & arg = args.children.at(1); @@ -693,7 +849,8 @@ static SharedPtr interpretSubquery( * Так как результат этого поздапроса - ещё не результат всего запроса. * Вместо этого работают ограничения * max_rows_in_set, max_bytes_in_set, set_overflow_mode, - * max_rows_in_join, max_bytes_in_join, join_overflow_mode. + * max_rows_in_join, max_bytes_in_join, join_overflow_mode, + * которые проверяются отдельно (в объектах Set, Join). */ Context subquery_context = context; Settings subquery_settings = context.getSettings(); @@ -740,10 +897,6 @@ static SharedPtr interpretSubquery( void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name) { - /// Сгенерируем имя для внешней таблицы. - while (context.tryGetExternalTable("_data" + toString(external_table_id))) - ++external_table_id; - if (const ASTIdentifier * table = typeid_cast(&*subquery_or_table_name)) { /// Если это уже внешняя таблица, ничего заполять не нужно. Просто запоминаем ее наличие. @@ -754,13 +907,20 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name) } } + /// Сгенерируем имя для внешней таблицы. + String external_table_name = "_data" + toString(external_table_id); + while (context.tryGetExternalTable(external_table_name) + || external_tables.count(external_table_name)) + { + ++external_table_id; + external_table_name = "_data" + toString(external_table_id); + } + SharedPtr interpreter = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1); Block sample = interpreter->getSampleBlock(); NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList()); - String external_table_name = "_data" + toString(external_table_id); - /** Заменяем подзапрос на имя временной таблицы. * Именно в таком виде, запрос отправится на удалённый сервер. * На удалённый сервер отправится эта временная таблица, и на его стороне, @@ -1213,7 +1373,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); NameSet joined_columns; joined_columns.insert(result_name); - actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns)); + actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false)); } return; @@ -1221,7 +1381,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl if (node->kind == ASTFunction::FUNCTION) { - if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn") + if (functionIsInOrGlobalInOperator(node->name)) { if (!no_subqueries) { @@ -1510,7 +1670,7 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio result_columns.insert(result_source.first); } - actions->add(ExpressionAction::arrayJoin(result_columns)); + actions->add(ExpressionAction::arrayJoin(result_columns, select_query->array_join_is_left)); } bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types) diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 0cdb6713c7f..0e4525efb9a 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -64,7 +64,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) try { auto dict_ptr = failed_dictionary.second.dict->clone(); - if (dict_ptr->getCreationException()) + if (const auto exception_ptr = dict_ptr->getCreationException()) { /// recalculate next attempt time std::uniform_int_distribution distribution( @@ -72,10 +72,11 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) failed_dictionary.second.next_attempt_time = std::chrono::system_clock::now() + std::chrono::seconds{ - std::min(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine)) - }; + std::min(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine))}; ++failed_dictionary.second.error_count; + + std::rethrow_exception(exception_ptr); } else { @@ -99,7 +100,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) } catch (...) { - LOG_ERROR(log, "Failed reloading " << name << " dictionary due to unexpected error"); + tryLogCurrentException(log, "Failed reloading '" + name + "' dictionary"); } } @@ -114,6 +115,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) try { + /// Если словарь не удалось ни разу загрузить или даже не удалось инициализировать из конфига. if (!dictionary.second.dict) continue; @@ -144,6 +146,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) { /// create new version of dictionary auto new_version = current->clone(); + + if (const auto exception_ptr = new_version->getCreationException()) + std::rethrow_exception(exception_ptr); + dictionary.second.dict->set(new_version.release()); } } @@ -155,25 +161,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) { dictionary.second.exception = std::current_exception(); - try - { - throw; - } - catch (const Poco::Exception & e) - { - LOG_ERROR(log, "Cannot update external dictionary '" << name - << "'! You must resolve this manually. " << e.displayText()); - } - catch (const std::exception & e) - { - LOG_ERROR(log, "Cannot update external dictionary '" << name - << "'! You must resolve this manually. " << e.what()); - } - catch (...) - { - LOG_ERROR(log, "Cannot update external dictionary '" << name - << "'! You must resolve this manually."); - } + tryLogCurrentException(log, "Cannot update external dictionary '" + name + "', leaving old version"); } } } @@ -235,6 +223,8 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const throw std::runtime_error{"Overriding dictionary from file " + dict_it->second.origin}; auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context); + + /// Если словарь не удалось загрузить. if (const auto exception_ptr = dict_ptr->getCreationException()) { const auto failed_dict_it = failed_dictionaries.find(name); @@ -292,6 +282,9 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const { if (!name.empty()) { + /// Если для словаря не удалось загрузить данные или даже не удалось инициализировать из конфига. + /// - всё-равно вставляем информацию в dictionaries, с нулевым указателем dict. + const std::lock_guard lock{dictionaries_mutex}; const auto exception_ptr = std::current_exception(); @@ -302,25 +295,7 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const dict_it->second.exception = exception_ptr; } - try - { - throw; - } - catch (const Poco::Exception & e) - { - LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name - << "'! You must resolve this manually. " << e.displayText()); - } - catch (const std::exception & e) - { - LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name - << "'! You must resolve this manually. " << e.what()); - } - catch (...) - { - LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name - << "'! You must resolve this manually."); - } + tryLogCurrentException(log, "Cannot create external dictionary '" + name + "' from config path " + config_path); /// propagate exception if (throw_on_error) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 662eda59e69..a9eac811dbe 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include @@ -42,7 +42,7 @@ InterpreterCreateQuery::InterpreterCreateQuery(ASTPtr query_ptr_, Context & cont } -void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) +BlockIO InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) { String path = context.getPath(); String current_database = context.getCurrentDatabase(); @@ -81,7 +81,7 @@ void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) if (!create.if_not_exists || !context.isDatabaseExist(database_name)) context.addDatabase(database_name); - return; + return {}; } SharedPtr interpreter_select; @@ -118,7 +118,7 @@ void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) if (context.isTableExist(database_name, table_name)) { if (create.if_not_exists) - return; + return {}; else throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } @@ -251,9 +251,16 @@ void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) /// Если запрос CREATE SELECT, то вставим в таблицу данные if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate)) { - BlockInputStreamPtr from = new MaterializingBlockInputStream(interpreter_select->execute().in); - copyData(*from, *res->write(query_ptr)); + BlockIO io; + io.in_sample = select_sample; + io.in = new NullAndDoCopyBlockInputStream( + new MaterializingBlockInputStream(interpreter_select->execute().in), + res->write(query_ptr)); + + return io; } + + return {}; } InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(ASTPtr expression_list) diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index cd9da42f770..06bc5709614 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -100,6 +100,7 @@ BlockIO InterpreterInsertQuery::execute() InterpreterSelectQuery interpreter_select{query.select, context}; BlockInputStreamPtr in{interpreter_select.execute().in}; res.in = new NullAndDoCopyBlockInputStream{in, out}; + res.in_sample = interpreter_select.getSampleBlock(); } return res; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 6c0d79c6474..e5b73b21f66 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -330,9 +331,6 @@ BlockIO InterpreterSelectQuery::execute() /// Ограничения на результат, квота на результат, а также колбек для прогресса. if (IProfilingBlockInputStream * stream = dynamic_cast(&*streams[0])) { - stream->setProgressCallback(context.getProgressCallback()); - stream->setProcessListElement(context.getProcessListElement()); - /// Ограничения действуют только на конечный результат. if (to_stage == QueryProcessingStage::Complete) { @@ -406,7 +404,7 @@ void InterpreterSelectQuery::executeSingleQuery() bool has_having = false; bool has_order_by = false; - ExpressionActionsPtr before_join; + ExpressionActionsPtr before_join; /// включая JOIN ExpressionActionsPtr before_where; ExpressionActionsPtr before_aggregation; ExpressionActionsPtr before_having; @@ -859,14 +857,38 @@ void InterpreterSelectQuery::executeAggregation(ExpressionActionsPtr expression, void InterpreterSelectQuery::executeMergeAggregated(bool overflow_row, bool final) { - /// Склеим несколько источников в один - executeUnion(); - - /// Теперь объединим агрегированные блоки Names key_names; AggregateDescriptions aggregates; query_analyzer->getAggregateInfo(key_names, aggregates); - streams[0] = new MergingAggregatedBlockInputStream(streams[0], key_names, aggregates, overflow_row, final, original_max_threads); + + /** Есть два режима распределённой агрегации. + * + * 1. В разных потоках читать из удалённых серверов блоки. + * Сохранить все блоки в оперативку. Объединить блоки. + * Если агрегация двухуровневая - распараллелить по номерам корзин. + * + * 2. В одном потоке читать по очереди блоки с разных серверов. + * В оперативке хранится только по одному блоку с каждого сервера. + * Если агрегация двухуровневая - последовательно объединяем блоки каждого следующего уровня. + * + * Второй вариант расходует меньше памяти (до 256 раз меньше) + * в случае двухуровневой агрегации, которая используется для больших результатов после GROUP BY, + * но при этом может работать медленнее. + */ + + if (!settings.distributed_aggregation_memory_efficient) + { + /// Склеим несколько источников в один, распараллеливая работу. + executeUnion(); + + /// Теперь объединим агрегированные блоки + streams[0] = new MergingAggregatedBlockInputStream(streams[0], key_names, aggregates, overflow_row, final, original_max_threads); + } + else + { + streams[0] = new MergingAggregatedMemoryEfficientBlockInputStream(streams, key_names, aggregates, overflow_row, final); + streams.resize(1); + } } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 6912b9f4945..126b175c6bd 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -166,6 +166,15 @@ static std::tuple executeQueryImpl( /// Держим элемент списка процессов до конца обработки запроса. res.process_list_entry = process_list_entry; + if (res.in) + { + if (IProfilingBlockInputStream * stream = dynamic_cast(res.in.get())) + { + stream->setProgressCallback(context.getProgressCallback()); + stream->setProcessListElement(context.getProcessListElement()); + } + } + quota.addQuery(current_time); /// Всё, что связано с логом запросов. diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 07c4be82267..76652d43a7d 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -41,7 +41,7 @@ static void executeCreateQuery(const String & query, Context & context, const St { if (const auto id = dynamic_cast(ast_create_query.storage.get())) { - if (id->name == "TinyLog") + if (id->name == "TinyLog" || id->name == "StripeLog") { tryLogCurrentException(__PRETTY_FUNCTION__); return; diff --git a/dbms/src/Interpreters/sortBlock.cpp b/dbms/src/Interpreters/sortBlock.cpp index f1b7405bd4f..aebc60ff1ea 100644 --- a/dbms/src/Interpreters/sortBlock.cpp +++ b/dbms/src/Interpreters/sortBlock.cpp @@ -147,28 +147,38 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit) } -void stableSortBlock(Block & block, const SortDescription & description) +void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation) { if (!block) return; size_t size = block.rows(); - IColumn::Permutation perm(size); + out_permutation.resize(size); for (size_t i = 0; i < size; ++i) - perm[i] = i; + out_permutation[i] = i; ColumnsWithSortDescriptions columns_with_sort_desc; for (size_t i = 0, size = description.size(); i < size; ++i) { - IColumn * column = !description[i].column_name.empty() + const IColumn * column = !description[i].column_name.empty() ? block.getByName(description[i].column_name).column : block.getByPosition(description[i].column_number).column; columns_with_sort_desc.push_back(std::make_pair(column, description[i])); } - std::stable_sort(perm.begin(), perm.end(), PartialSortingLess(columns_with_sort_desc)); + std::stable_sort(out_permutation.begin(), out_permutation.end(), PartialSortingLess(columns_with_sort_desc)); +} + + +void stableSortBlock(Block & block, const SortDescription & description) +{ + if (!block) + return; + + IColumn::Permutation perm; + stableGetPermutation(block, description, perm); size_t columns = block.columns(); for (size_t i = 0; i < columns; ++i) diff --git a/dbms/src/Parsers/ASTFunction.cpp b/dbms/src/Parsers/ASTFunction.cpp new file mode 100644 index 00000000000..8a4b1083bc2 --- /dev/null +++ b/dbms/src/Parsers/ASTFunction.cpp @@ -0,0 +1,186 @@ +#include +#include + + +namespace DB +{ + +void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + FormatStateStacked nested_need_parens = frame; + FormatStateStacked nested_dont_need_parens = frame; + nested_need_parens.need_parens = true; + nested_dont_need_parens.need_parens = false; + + /// Стоит ли записать эту функцию в виде оператора? + bool written = false; + if (arguments && !parameters) + { + if (arguments->children.size() == 1) + { + const char * operators[] = + { + "negate", "-", + "not", "NOT ", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (0 == strcmp(name.c_str(), func[0])) + { + settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : ""); + + /** Особо дурацкий случай. Если у нас унарный минус перед литералом, являющимся отрицательным числом: + * "-(-1)" или "- -1", то это нельзя форматировать как --1, так как это будет воспринято как комментарий. + * Вместо этого, добавим пробел. + * PS. Нельзя просто попросить добавить скобки - см. formatImpl для ASTLiteral. + */ + if (name == "negate" && typeid_cast(&*arguments->children[0])) + settings.ostr << ' '; + + arguments->formatImpl(settings, state, nested_need_parens); + written = true; + } + } + } + + /** need_parens - нужны ли скобки вокруг выражения с оператором. + * Они нужны, только если это выражение входит в другое выражение с оператором. + */ + + if (!written && arguments->children.size() == 2) + { + const char * operators[] = + { + "multiply", " * ", + "divide", " / ", + "modulo", " % ", + "plus", " + ", + "minus", " - ", + "notEquals", " != ", + "lessOrEquals", " <= ", + "greaterOrEquals", " >= ", + "less", " < ", + "greater", " > ", + "equals", " = ", + "like", " LIKE ", + "notLike", " NOT LIKE ", + "in", " IN ", + "notIn", " NOT IN ", + "globalIn", " GLOBAL IN ", + "globalNotIn", " GLOBAL NOT IN ", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (0 == strcmp(name.c_str(), func[0])) + { + if (frame.need_parens) + settings.ostr << '('; + arguments->children[0]->formatImpl(settings, state, nested_need_parens); + settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : ""); + arguments->children[1]->formatImpl(settings, state, nested_need_parens); + if (frame.need_parens) + settings.ostr << ')'; + written = true; + } + } + + if (!written && 0 == strcmp(name.c_str(), "arrayElement")) + { + arguments->children[0]->formatImpl(settings, state, nested_need_parens); + settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : ""); + arguments->children[1]->formatImpl(settings, state, nested_need_parens); + settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : ""); + written = true; + } + + if (!written && 0 == strcmp(name.c_str(), "tupleElement")) + { + arguments->children[0]->formatImpl(settings, state, nested_need_parens); + settings.ostr << (settings.hilite ? hilite_operator : "") << "." << (settings.hilite ? hilite_none : ""); + arguments->children[1]->formatImpl(settings, state, nested_need_parens); + written = true; + } + } + + if (!written && arguments->children.size() >= 2) + { + const char * operators[] = + { + "and", " AND ", + "or", " OR ", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (0 == strcmp(name.c_str(), func[0])) + { + if (frame.need_parens) + settings.ostr << '('; + for (size_t i = 0; i < arguments->children.size(); ++i) + { + if (i != 0) + settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : ""); + arguments->children[i]->formatImpl(settings, state, nested_need_parens); + } + if (frame.need_parens) + settings.ostr << ')'; + written = true; + } + } + } + + if (!written && arguments->children.size() >= 1 && 0 == strcmp(name.c_str(), "array")) + { + settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : ""); + for (size_t i = 0; i < arguments->children.size(); ++i) + { + if (i != 0) + settings.ostr << ", "; + arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); + } + settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : ""); + written = true; + } + + if (!written && arguments->children.size() >= 2 && 0 == strcmp(name.c_str(), "tuple")) + { + settings.ostr << (settings.hilite ? hilite_operator : "") << '(' << (settings.hilite ? hilite_none : ""); + for (size_t i = 0; i < arguments->children.size(); ++i) + { + if (i != 0) + settings.ostr << ", "; + arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); + } + settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : ""); + written = true; + } + } + + if (!written) + { + settings.ostr << (settings.hilite ? hilite_function : "") << name; + + if (parameters) + { + settings.ostr << '(' << (settings.hilite ? hilite_none : ""); + parameters->formatImpl(settings, state, nested_dont_need_parens); + settings.ostr << (settings.hilite ? hilite_function : "") << ')'; + } + + if (arguments) + { + settings.ostr << '(' << (settings.hilite ? hilite_none : ""); + arguments->formatImpl(settings, state, nested_dont_need_parens); + settings.ostr << (settings.hilite ? hilite_function : "") << ')'; + } + + settings.ostr << (settings.hilite ? hilite_none : ""); + } +} + +} diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index ff6f53d5a4e..fbb22ad0dd0 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -1,5 +1,7 @@ +#include #include + namespace DB { @@ -217,5 +219,154 @@ const IAST * ASTSelectQuery::getFormat() const return query->format.get(); } + +void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const +{ + frame.current_select = this; + frame.need_parens = false; + std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); + + s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (distinct ? "DISTINCT " : "") << (s.hilite ? hilite_none : ""); + + s.one_line + ? select_expression_list->formatImpl(s, state, frame) + : typeid_cast(*select_expression_list).formatImplMultiline(s, state, frame); + + if (table) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FROM " << (s.hilite ? hilite_none : ""); + if (database) + { + database->formatImpl(s, state, frame); + s.ostr << "."; + } + + if (typeid_cast(&*table)) + { + if (s.one_line) + s.ostr << " ("; + else + s.ostr << "\n" << indent_str << "(\n"; + + FormatStateStacked frame_with_indent = frame; + ++frame_with_indent.indent; + table->formatImpl(s, state, frame_with_indent); + + if (s.one_line) + s.ostr << ")"; + else + s.ostr << "\n" << indent_str << ")"; + } + else + table->formatImpl(s, state, frame); + } + + if (final) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FINAL" << (s.hilite ? hilite_none : ""); + } + + if (sample_size) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SAMPLE " << (s.hilite ? hilite_none : ""); + sample_size->formatImpl(s, state, frame); + } + + if (array_join_expression_list) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str + << (array_join_is_left ? "LEFT " : "") << "ARRAY JOIN " << (s.hilite ? hilite_none : ""); + + s.one_line + ? array_join_expression_list->formatImpl(s, state, frame) + : typeid_cast(*array_join_expression_list).formatImplMultiline(s, state, frame); + } + + if (join) + { + s.ostr << " "; + join->formatImpl(s, state, frame); + } + + if (prewhere_expression) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "PREWHERE " << (s.hilite ? hilite_none : ""); + prewhere_expression->formatImpl(s, state, frame); + } + + if (where_expression) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "WHERE " << (s.hilite ? hilite_none : ""); + where_expression->formatImpl(s, state, frame); + } + + if (group_expression_list) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : ""); + s.one_line + ? group_expression_list->formatImpl(s, state, frame) + : typeid_cast(*group_expression_list).formatImplMultiline(s, state, frame); + } + + if (group_by_with_totals) + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : ""); + + if (having_expression) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "HAVING " << (s.hilite ? hilite_none : ""); + having_expression->formatImpl(s, state, frame); + } + + if (order_expression_list) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : ""); + s.one_line + ? order_expression_list->formatImpl(s, state, frame) + : typeid_cast(*order_expression_list).formatImplMultiline(s, state, frame); + } + + if (limit_length) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "LIMIT " << (s.hilite ? hilite_none : ""); + if (limit_offset) + { + limit_offset->formatImpl(s, state, frame); + s.ostr << ", "; + } + limit_length->formatImpl(s, state, frame); + } + + if (settings) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : ""); + + const ASTSetQuery & ast_set = typeid_cast(*settings); + for (ASTSetQuery::Changes::const_iterator it = ast_set.changes.begin(); it != ast_set.changes.end(); ++it) + { + if (it != ast_set.changes.begin()) + s.ostr << ", "; + + s.ostr << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value); + } + } + + if (format) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FORMAT " << (s.hilite ? hilite_none : ""); + format->formatImpl(s, state, frame); + } + + if (next_union_all) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "UNION ALL " << s.nl_or_ws << (s.hilite ? hilite_none : ""); + + // NOTE Мы можем безопасно применить static_cast вместо typeid_cast, потому что знаем, что в цепочке UNION ALL + // имеются только деревья типа SELECT. + const ASTSelectQuery & next_ast = static_cast(*next_union_all); + + next_ast.formatImpl(s, state, frame); + } +} + }; diff --git a/dbms/src/Parsers/ASTWithAlias.cpp b/dbms/src/Parsers/ASTWithAlias.cpp new file mode 100644 index 00000000000..97016f7eb17 --- /dev/null +++ b/dbms/src/Parsers/ASTWithAlias.cpp @@ -0,0 +1,33 @@ +#include + +namespace DB +{ + +void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + if (!alias.empty()) + { + /// Если мы уже ранее вывели этот узел в другом месте запроса, то теперь достаточно вывести лишь алиас. + if (!state.printed_asts_with_alias.emplace(frame.current_select, alias).second) + { + WriteBufferFromOStream wb(settings.ostr, 32); + writeProbablyBackQuotedString(alias, wb); + return; + } + } + + /// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна. + if (frame.need_parens && !alias.empty()) + settings.ostr <<'('; + + formatImplWithoutAlias(settings, state, frame); + + if (!alias.empty()) + { + writeAlias(alias, settings.ostr, settings.hilite); + if (frame.need_parens) + settings.ostr <<')'; + } +} + +} diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 1209af32578..e58849423e9 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -556,12 +556,8 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & { String alias_name = typeid_cast(*alias_node).name; - if (ASTFunction * func = typeid_cast(&*node)) - func->alias = alias_name; - else if (ASTIdentifier * ident = typeid_cast(&*node)) - ident->alias = alias_name; - else if (ASTLiteral * lit = typeid_cast(&*node)) - lit->alias = alias_name; + if (ASTWithAlias * ast_with_alias = dynamic_cast(node.get())) + ast_with_alias->alias = alias_name; else { expected = "alias cannot be here"; diff --git a/dbms/src/Parsers/IAST.cpp b/dbms/src/Parsers/IAST.cpp new file mode 100644 index 00000000000..2c6f0b8a3e1 --- /dev/null +++ b/dbms/src/Parsers/IAST.cpp @@ -0,0 +1,39 @@ +#include +#include + + +namespace DB +{ + +const char * IAST::hilite_keyword = "\033[1m"; +const char * IAST::hilite_identifier = "\033[0;36m"; +const char * IAST::hilite_function = "\033[0;33m"; +const char * IAST::hilite_operator = "\033[1;33m"; +const char * IAST::hilite_alias = "\033[0;32m"; +const char * IAST::hilite_none = "\033[0m"; + + +/// Квотировать идентификатор обратными кавычками, если это требуется. +String backQuoteIfNeed(const String & x) +{ + String res(x.size(), '\0'); + { + WriteBufferFromString wb(res); + writeProbablyBackQuotedString(x, wb); + } + return res; +} + + +void IAST::writeAlias(const String & name, std::ostream & s, bool hilite) const +{ + s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : ""); + + WriteBufferFromOStream wb(s, 32); + writeProbablyBackQuotedString(name, wb); + wb.next(); + + s << (hilite ? hilite_none : ""); +} + +} diff --git a/dbms/src/Parsers/ParserJoin.cpp b/dbms/src/Parsers/ParserJoin.cpp index 4cb7aeb89bb..3c55cf82f0b 100644 --- a/dbms/src/Parsers/ParserJoin.cpp +++ b/dbms/src/Parsers/ParserJoin.cpp @@ -30,7 +30,7 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p ParserString s_using("USING", true, true); ParserNotEmptyExpressionList exp_list; - ParserSubquery subquery; + ParserWithOptionalAlias subquery(ParserPtr(new ParserSubquery)); ParserIdentifier identifier; ws.ignore(pos, end); @@ -91,10 +91,6 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p ws.ignore(pos, end); - /// Может быть указан алиас. На данный момент, он ничего не значит и не используется. - ParserAlias().ignore(pos, end); - ws.ignore(pos, end); - if (join->kind != ASTJoin::Cross) { if (!s_using.ignore(pos, end, max_parsed_pos, expected)) diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 97a171846d1..ee0908f185b 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -23,6 +23,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p ParserString s_select("SELECT", true, true); ParserString s_distinct("DISTINCT", true, true); ParserString s_from("FROM", true, true); + ParserString s_left("LEFT", true, true); ParserString s_array("ARRAY", true, true); ParserString s_join("JOIN", true, true); ParserString s_using("USING", true, true); @@ -166,8 +167,22 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p if (!parse_final_and_sample()) return false; - /// ARRAY JOIN expr list - if (s_array.ignore(pos, end, max_parsed_pos, expected)) + /// [LEFT] ARRAY JOIN expr list + Pos saved_pos = pos; + bool has_array_join = false; + if (s_left.ignore(pos, end, max_parsed_pos, expected) && ws.ignore(pos, end) && s_array.ignore(pos, end, max_parsed_pos, expected)) + { + select_query->array_join_is_left = true; + has_array_join = true; + } + else + { + pos = saved_pos; + if (s_array.ignore(pos, end, max_parsed_pos, expected)) + has_array_join = true; + } + + if (has_array_join) { ws.ignore(pos, end); @@ -182,7 +197,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p ws.ignore(pos, end); } - /// [GLOBAL] ANY|ALL INNER|LEFT JOIN (subquery) USING tuple + /// [GLOBAL] [ANY|ALL] INNER|LEFT|RIGHT|FULL|CROSS [OUTER] JOIN (subquery)|table_name USING tuple join.parse(pos, end, select_query->join, max_parsed_pos, expected); if (!parse_final_and_sample()) diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index 3c1a0e7b62d..7966b0db491 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -1,943 +1,9 @@ -#include - -#include - -#include -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -//#include - #include namespace DB { - -static const char * hilite_keyword = "\033[1m"; -static const char * hilite_identifier = "\033[0;36m"; -static const char * hilite_function = "\033[0;33m"; -static const char * hilite_operator = "\033[1;33m"; -static const char * hilite_alias = "\033[0;32m"; -static const char * hilite_none = "\033[0m"; - - -/// Квотировать идентификатор обратными кавычками, если это требуется. -String backQuoteIfNeed(const String & x) -{ - String res(x.size(), '\0'); - { - WriteBufferFromString wb(res); - writeProbablyBackQuotedString(x, wb); - } - return res; -} - - -String hightlight(const String & keyword, const String & color_sequence, const bool hilite) -{ - return hilite ? color_sequence + keyword + hilite_none : keyword; -} - - -void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it) - { - if (it != ast.children.begin()) - s << ", "; - - formatAST(**it, s, indent, hilite, one_line, need_parens); - } -} - -/** Вывести список выражений в секциях запроса SELECT - по одному выражению на строку. - */ -static void formatExpressionListMultiline(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite) -{ - std::string indent_str = "\n" + std::string(4 * (indent + 1), ' '); - - for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it) - { - if (it != ast.children.begin()) - s << ", "; - - if (ast.children.size() > 1) - s << indent_str; - - formatAST(**it, s, indent + 1, hilite, false); - } -} - - -void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string nl_or_nothing = one_line ? "" : "\n"; - - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - - s << (hilite ? hilite_keyword : "") << indent_str << "SELECT " << (ast.distinct ? "DISTINCT " : "") << (hilite ? hilite_none : ""); - one_line - ? formatAST(*ast.select_expression_list, s, indent, hilite, one_line) - : formatExpressionListMultiline(typeid_cast(*ast.select_expression_list), s, indent, hilite); - - if (ast.table) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FROM " << (hilite ? hilite_none : ""); - if (ast.database) - { - formatAST(*ast.database, s, indent, hilite, one_line); - s << "."; - } - - if (typeid_cast(&*ast.table)) - { - if (one_line) - s << " ("; - else - s << "\n" << indent_str << "(\n"; - - formatAST(*ast.table, s, indent + 1, hilite, one_line); - - if (one_line) - s << ")"; - else - s << "\n" << indent_str << ")"; - } - else - formatAST(*ast.table, s, indent, hilite, one_line); - } - - if (ast.final) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FINAL" << (hilite ? hilite_none : ""); - } - - if (ast.sample_size) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "SAMPLE " << (hilite ? hilite_none : ""); - formatAST(*ast.sample_size, s, indent, hilite, one_line); - } - - if (ast.array_join_expression_list) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "ARRAY JOIN " << (hilite ? hilite_none : ""); - one_line - ? formatAST(*ast.array_join_expression_list, s, indent, hilite, one_line) - : formatExpressionListMultiline(typeid_cast(*ast.array_join_expression_list), s, indent, hilite); - } - - if (ast.join) - { - s << " "; - formatAST(*ast.join, s, indent, hilite, one_line); - } - - if (ast.prewhere_expression) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "PREWHERE " << (hilite ? hilite_none : ""); - formatAST(*ast.prewhere_expression, s, indent, hilite, one_line); - } - - if (ast.where_expression) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "WHERE " << (hilite ? hilite_none : ""); - formatAST(*ast.where_expression, s, indent, hilite, one_line); - } - - if (ast.group_expression_list) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "GROUP BY " << (hilite ? hilite_none : ""); - one_line - ? formatAST(*ast.group_expression_list, s, indent, hilite, one_line) - : formatExpressionListMultiline(typeid_cast(*ast.group_expression_list), s, indent, hilite); - } - - if (ast.group_by_with_totals) - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << (one_line ? "" : " ") << "WITH TOTALS" << (hilite ? hilite_none : ""); - - if (ast.having_expression) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "HAVING " << (hilite ? hilite_none : ""); - formatAST(*ast.having_expression, s, indent, hilite, one_line); - } - - if (ast.order_expression_list) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "ORDER BY " << (hilite ? hilite_none : ""); - one_line - ? formatAST(*ast.order_expression_list, s, indent, hilite, one_line) - : formatExpressionListMultiline(typeid_cast(*ast.order_expression_list), s, indent, hilite); - } - - if (ast.limit_length) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "LIMIT " << (hilite ? hilite_none : ""); - if (ast.limit_offset) - { - formatAST(*ast.limit_offset, s, indent, hilite, one_line); - s << ", "; - } - formatAST(*ast.limit_length, s, indent, hilite, one_line); - } - - if (ast.settings) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "SETTINGS " << (hilite ? hilite_none : ""); - - const ASTSetQuery & ast_set = typeid_cast(*ast.settings); - for (ASTSetQuery::Changes::const_iterator it = ast_set.changes.begin(); it != ast_set.changes.end(); ++it) - { - if (it != ast_set.changes.begin()) - s << ", "; - - s << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value); - } - } - - if (ast.format) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FORMAT " << (hilite ? hilite_none : ""); - formatAST(*ast.format, s, indent, hilite, one_line); - } - - if (ast.next_union_all) - { - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "UNION ALL " << nl_or_ws << (hilite ? hilite_none : ""); - - // NOTE Мы можем безопасно применить static_cast вместо typeid_cast, потому что знаем, что в цепочке UNION ALL - // имеются только деревья типа SELECT. - const ASTSelectQuery & next_ast = static_cast(*ast.next_union_all); - - formatAST(next_ast, s, indent, hilite, one_line, need_parens); - } -} - -void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_nothing = one_line ? "" : "\n"; - - s << nl_or_nothing << indent_str << "(" << nl_or_nothing; - formatAST(*ast.children[0], s, indent + 1, hilite, one_line); - s << nl_or_nothing << indent_str << ")"; -} - -void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string nl_or_ws = one_line ? " " : "\n"; - - if (!ast.database.empty() && ast.table.empty()) - { - s << (hilite ? hilite_keyword : "") << (ast.attach ? "ATTACH DATABASE " : "CREATE DATABASE ") << (ast.if_not_exists ? "IF NOT EXISTS " : "") << (hilite ? hilite_none : "") - << backQuoteIfNeed(ast.database); - return; - } - - { - std::string what = "TABLE"; - if (ast.is_view) - what = "VIEW"; - if (ast.is_materialized_view) - what = "MATERIALIZED VIEW"; - - s << (hilite ? hilite_keyword : "") << (ast.attach ? "ATTACH " : "CREATE ") << (ast.is_temporary ? "TEMPORARY " : "") << what << " " << (ast.if_not_exists ? "IF NOT EXISTS " : "") << (hilite ? hilite_none : "") - << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); - } - - if (!ast.as_table.empty()) - { - s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_none : "") - << (!ast.as_database.empty() ? backQuoteIfNeed(ast.as_database) + "." : "") << backQuoteIfNeed(ast.as_table); - } - - if (ast.columns) - { - s << (one_line ? " (" : "\n("); - formatAST(*ast.columns, s, indent + 1, hilite, one_line); - s << (one_line ? ")" : "\n)"); - } - - if (ast.storage && !ast.is_materialized_view && !ast.is_view) - { - s << (hilite ? hilite_keyword : "") << " ENGINE" << (hilite ? hilite_none : "") << " = "; - formatAST(*ast.storage, s, indent, hilite, one_line); - } - - if (ast.inner_storage) - { - s << (hilite ? hilite_keyword : "") << " ENGINE" << (hilite ? hilite_none : "") << " = "; - formatAST(*ast.inner_storage, s, indent, hilite, one_line); - } - - if (ast.is_populate) - { - s << (hilite ? hilite_keyword : "") << " POPULATE" << (hilite ? hilite_none : ""); - } - - if (ast.select) - { - s << (hilite ? hilite_keyword : "") << " AS" << nl_or_ws << (hilite ? hilite_none : ""); - formatAST(*ast.select, s, indent, hilite, one_line); - } -} - -void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - if (ast.table.empty() && !ast.database.empty()) - { - s << (hilite ? hilite_keyword : "") << (ast.detach ? "DETACH DATABASE " : "DROP DATABASE ") << (ast.if_exists ? "IF EXISTS " : "") << (hilite ? hilite_none : "") << backQuoteIfNeed(ast.database); - return; - } - - s << (hilite ? hilite_keyword : "") << (ast.detach ? "DETACH TABLE " : "DROP TABLE ") << (ast.if_exists ? "IF EXISTS " : "") << (hilite ? hilite_none : "") - << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); -} - -void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (hilite ? hilite_none : "") - << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); -} - -void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << name << " " << (hilite ? hilite_none : "") - << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); - - if (ast.format) - { - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FORMAT " << (hilite ? hilite_none : ""); - formatAST(*ast.format, s, indent, hilite, one_line); - } -} - -void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - formatAST(static_cast(ast), "EXISTS TABLE", s, indent, hilite, one_line, false); -} - -void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - formatAST(static_cast(ast), "DESCRIBE TABLE", s, indent, hilite, one_line, false); -} - -void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - formatAST(static_cast(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line, false); -} - -void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "RENAME TABLE " << (hilite ? hilite_none : ""); - - for (ASTRenameQuery::Elements::const_iterator it = ast.elements.begin(); it != ast.elements.end(); ++it) - { - if (it != ast.elements.begin()) - s << ", "; - - s << (!it->from.database.empty() ? backQuoteIfNeed(it->from.database) + "." : "") << backQuoteIfNeed(it->from.table) - << (hilite ? hilite_keyword : "") << " TO " << (hilite ? hilite_none : "") - << (!it->to.database.empty() ? backQuoteIfNeed(it->to.database) + "." : "") << backQuoteIfNeed(it->to.table); - } -} - -void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "SET " << (ast.global ? "GLOBAL " : "") << (hilite ? hilite_none : ""); - - for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it) - { - if (it != ast.changes.begin()) - s << ", "; - - s << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value); - } -} - -void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - if (ast.databases) - { - s << (hilite ? hilite_keyword : "") << "SHOW DATABASES" << (hilite ? hilite_none : ""); - } - else - { - s << (hilite ? hilite_keyword : "") << "SHOW TABLES" << (hilite ? hilite_none : ""); - - if (!ast.from.empty()) - s << (hilite ? hilite_keyword : "") << " FROM " << (hilite ? hilite_none : "") - << backQuoteIfNeed(ast.from); - - if (!ast.like.empty()) - s << (hilite ? hilite_keyword : "") << " LIKE " << (hilite ? hilite_none : "") - << mysqlxx::quote << ast.like; - } - - if (ast.format) - { - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FORMAT " << (hilite ? hilite_none : ""); - formatAST(*ast.format, s, indent, hilite, one_line); - } -} - -void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "USE " << (hilite ? hilite_none : "") << backQuoteIfNeed(ast.database); - return; -} - -void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "SHOW PROCESSLIST" << (hilite ? hilite_none : ""); - return; -} - -void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "INSERT INTO " << (hilite ? hilite_none : "") - << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); - - if (!ast.insert_id.empty()) - s << (hilite ? hilite_keyword : "") << " ID = " << (hilite ? hilite_none : "") - << mysqlxx::quote << ast.insert_id; - - if (ast.columns) - { - s << " ("; - formatAST(*ast.columns, s, indent, hilite, one_line); - s << ")"; - } - - if (ast.select) - { - s << " "; - formatAST(*ast.select, s, indent, hilite, one_line); - } - else - { - if (!ast.format.empty()) - { - s << (hilite ? hilite_keyword : "") << " FORMAT " << (hilite ? hilite_none : "") << ast.format; - } - else - { - s << (hilite ? hilite_keyword : "") << " VALUES" << (hilite ? hilite_none : ""); - } - } -} - -static void writeAlias(const String & name, std::ostream & s, bool hilite, bool one_line) -{ - s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : ""); - - WriteBufferFromOStream wb(s, 32); - writeProbablyBackQuotedString(name, wb); - wb.next(); - - s << (hilite ? hilite_none : ""); -} - -void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - /// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна. - if (need_parens && !ast.alias.empty()) - s << '('; - - /// Стоит ли записать эту функцию в виде оператора? - bool written = false; - if (ast.arguments && !ast.parameters) - { - if (ast.arguments->children.size() == 1) - { - const char * operators[] = - { - "negate", "-", - "not", "NOT ", - nullptr - }; - - for (const char ** func = operators; *func; func += 2) - { - if (0 == strcmp(ast.name.c_str(), func[0])) - { - s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : ""); - - /** Особо дурацкий случай. Если у нас унарный минус перед литералом, являющимся отрицательным числом: - * "-(-1)" или "- -1", то это нельзя форматировать как --1, так как это будет воспринято как комментарий. - * Вместо этого, добавим пробел. - * PS. Нельзя просто попросить добавить скобки - см. formatAST для ASTLiteral. - */ - if (ast.name == "negate" && typeid_cast(&*ast.arguments->children[0])) - s << ' '; - - formatAST(*ast.arguments, s, indent, hilite, one_line, true); - written = true; - } - } - } - - /** need_parens - нужны ли скобки вокруг выражения с оператором. - * Они нужны, только если это выражение входит в другое выражение с оператором. - */ - - if (!written && ast.arguments->children.size() == 2) - { - const char * operators[] = - { - "multiply", " * ", - "divide", " / ", - "modulo", " % ", - "plus", " + ", - "minus", " - ", - "notEquals", " != ", - "lessOrEquals", " <= ", - "greaterOrEquals", " >= ", - "less", " < ", - "greater", " > ", - "equals", " = ", - "like", " LIKE ", - "notLike", " NOT LIKE ", - "in", " IN ", - "notIn", " NOT IN ", - "globalIn", " GLOBAL IN ", - "globalNotIn", " GLOBAL NOT IN ", - nullptr - }; - - for (const char ** func = operators; *func; func += 2) - { - if (0 == strcmp(ast.name.c_str(), func[0])) - { - if (need_parens) - s << '('; - formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true); - s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : ""); - formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true); - if (need_parens) - s << ')'; - written = true; - } - } - - if (!written && 0 == strcmp(ast.name.c_str(), "arrayElement")) - { - formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true); - s << (hilite ? hilite_operator : "") << '[' << (hilite ? hilite_none : ""); - formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true); - s << (hilite ? hilite_operator : "") << ']' << (hilite ? hilite_none : ""); - written = true; - } - - if (!written && 0 == strcmp(ast.name.c_str(), "tupleElement")) - { - formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true); - s << (hilite ? hilite_operator : "") << "." << (hilite ? hilite_none : ""); - formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true); - written = true; - } - } - - if (!written && ast.arguments->children.size() >= 2) - { - const char * operators[] = - { - "and", " AND ", - "or", " OR ", - nullptr - }; - - for (const char ** func = operators; *func; func += 2) - { - if (0 == strcmp(ast.name.c_str(), func[0])) - { - if (need_parens) - s << '('; - for (size_t i = 0; i < ast.arguments->children.size(); ++i) - { - if (i != 0) - s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : ""); - formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, true); - } - if (need_parens) - s << ')'; - written = true; - } - } - } - - if (!written && ast.arguments->children.size() >= 1 && 0 == strcmp(ast.name.c_str(), "array")) - { - s << (hilite ? hilite_operator : "") << '[' << (hilite ? hilite_none : ""); - for (size_t i = 0; i < ast.arguments->children.size(); ++i) - { - if (i != 0) - s << ", "; - formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false); - } - s << (hilite ? hilite_operator : "") << ']' << (hilite ? hilite_none : ""); - written = true; - } - - if (!written && ast.arguments->children.size() >= 2 && 0 == strcmp(ast.name.c_str(), "tuple")) - { - s << (hilite ? hilite_operator : "") << '(' << (hilite ? hilite_none : ""); - for (size_t i = 0; i < ast.arguments->children.size(); ++i) - { - if (i != 0) - s << ", "; - formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false); - } - s << (hilite ? hilite_operator : "") << ')' << (hilite ? hilite_none : ""); - written = true; - } - } - - if (!written) - { - s << (hilite ? hilite_function : "") << ast.name; - - if (ast.parameters) - { - s << '(' << (hilite ? hilite_none : ""); - formatAST(*ast.parameters, s, indent, hilite, one_line); - s << (hilite ? hilite_function : "") << ')'; - } - - if (ast.arguments) - { - s << '(' << (hilite ? hilite_none : ""); - formatAST(*ast.arguments, s, indent, hilite, one_line); - s << (hilite ? hilite_function : "") << ')'; - } - - s << (hilite ? hilite_none : ""); - } - - if (!ast.alias.empty()) - { - writeAlias(ast.alias, s, hilite, one_line); - if (need_parens) - s << ')'; - } -} - -void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - if (need_parens && !ast.alias.empty()) - s << '('; - - s << (hilite ? hilite_identifier : ""); - - WriteBufferFromOStream wb(s, 32); - writeProbablyBackQuotedString(ast.name, wb); - wb.next(); - - s << (hilite ? hilite_none : ""); - - if (!ast.alias.empty()) - { - writeAlias(ast.alias, s, hilite, one_line); - if (need_parens) - s << ')'; - } -} - -void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - if (need_parens && !ast.alias.empty()) - s << '('; - - s << apply_visitor(FieldVisitorToString(), ast.value); - - if (!ast.alias.empty()) - { - writeAlias(ast.alias, s, hilite, one_line); - if (need_parens) - s << ')'; - } -} - -void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - - s << nl_or_ws << indent_str << backQuoteIfNeed(ast.name) << " "; - formatAST(*ast.type, s, indent, hilite, one_line); -} - -void formatAST(const ASTColumnDeclaration & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - - s << nl_or_ws << indent_str << backQuoteIfNeed(ast.name); - if (ast.type) - { - s << ' '; - formatAST(*ast.type, s, indent, hilite, one_line); - } - - if (ast.default_expression) - { - s << ' ' << hightlight(ast.default_specifier, hilite_keyword, hilite) << ' '; - formatAST(*ast.default_expression, s, indent, hilite, one_line); - } -} - -void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << "*"; -} - -void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - formatAST(*ast.children.front(), s, indent, hilite, one_line); - s << (hilite ? hilite_keyword : "") << (ast.direction == -1 ? " DESC" : " ASC") << (hilite ? hilite_none : ""); - if (!ast.collator.isNull()) - { - s << (hilite ? hilite_keyword : "") << " COLLATE " << (hilite ? hilite_none : "") - << "'" << ast.collator->getLocale() << "'"; - } -} - -void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string nl_or_nothing = one_line ? "" : "\n"; - - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - - s << (hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (hilite ? hilite_none : ""); - - if (!ast.table.empty()) - { - if (!ast.database.empty()) - { - s << indent_str << ast.database; - s << "."; - } - s << indent_str << ast.table; - } - s << nl_or_ws; - - for (size_t i = 0; i < ast.parameters.size(); ++i) - { - const ASTAlterQuery::Parameters &p = ast.parameters[i]; - - if (p.type == ASTAlterQuery::ADD_COLUMN) - { - s << (hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (hilite ? hilite_none : ""); - formatAST(*p.col_decl, s, indent, hilite, true); - - /// AFTER - if (p.column) - { - s << (hilite ? hilite_keyword : "") << indent_str << " AFTER " << (hilite ? hilite_none : ""); - formatAST(*p.column, s, indent, hilite, one_line); - } - } - else if (p.type == ASTAlterQuery::DROP_COLUMN) - { - s << (hilite ? hilite_keyword : "") << indent_str << "DROP COLUMN " << (hilite ? hilite_none : ""); - formatAST(*p.column, s, indent, hilite, true); - } - else if (p.type == ASTAlterQuery::MODIFY_COLUMN) - { - s << (hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (hilite ? hilite_none : ""); - formatAST(*p.col_decl, s, indent, hilite, true); - } - else if (p.type == ASTAlterQuery::DROP_PARTITION) - { - s << (hilite ? hilite_keyword : "") << indent_str << (p.detach ? "DETACH" : "DROP") << " PARTITION " - << (hilite ? hilite_none : ""); - formatAST(*p.partition, s, indent, hilite, true); - } - else if (p.type == ASTAlterQuery::ATTACH_PARTITION) - { - s << (hilite ? hilite_keyword : "") << indent_str << "ATTACH " << (p.unreplicated ? "UNREPLICATED " : "") - << (p.part ? "PART " : "PARTITION ") << (hilite ? hilite_none : ""); - formatAST(*p.partition, s, indent, hilite, true); - } - else if (p.type == ASTAlterQuery::FETCH_PARTITION) - { - s << (hilite ? hilite_keyword : "") << indent_str << "FETCH " << (p.unreplicated ? "UNREPLICATED " : "") - << "PARTITION " << (hilite ? hilite_none : ""); - formatAST(*p.partition, s, indent, hilite, true); - s << (hilite ? hilite_keyword : "") << " FROM " << (hilite ? hilite_none : "") - << mysqlxx::quote << p.from; - } - else if (p.type == ASTAlterQuery::FREEZE_PARTITION) - { - s << (hilite ? hilite_keyword : "") << indent_str << "FREEZE PARTITION " << (hilite ? hilite_none : ""); - formatAST(*p.partition, s, indent, hilite, true); - } - else - throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - - std::string comma = (i < (ast.parameters.size() -1) ) ? "," : ""; - s << (hilite ? hilite_keyword : "") << indent_str << comma << (hilite ? hilite_none : ""); - - s << nl_or_ws; - } -} - -void formatAST(const ASTSet & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - /** Подготовленное множество. В пользовательских запросах такого не бывает, но такое бывает после промежуточных преобразований запроса. - * Выведем его не по-настоящему (это не будет корректным запросом, но покажет, что здесь было множество). - */ - s << (hilite ? hilite_keyword : "") - << "(...)" - << (hilite ? hilite_none : ""); -} - -void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : ""); - - if (ast.locality == ASTJoin::Global) - s << "GLOBAL "; - - if (ast.kind != ASTJoin::Cross) - s << (ast.strictness == ASTJoin::Any ? "ANY " : "ALL "); - - s << (ast.kind == ASTJoin::Inner ? "INNER " - : (ast.kind == ASTJoin::Left ? "LEFT " - : (ast.kind == ASTJoin::Right ? "RIGHT " - : (ast.kind == ASTJoin::Cross ? "CROSS " - : "FULL OUTER ")))); - - s << "JOIN " - << (hilite ? hilite_none : ""); - - formatAST(*ast.table, s, indent, hilite, one_line, need_parens); - - if (ast.kind != ASTJoin::Cross) - { - s << (hilite ? hilite_keyword : "") << " USING " << (hilite ? hilite_none : ""); - formatAST(*ast.using_expr_list, s, indent, hilite, one_line, need_parens); - } -} - -void formatAST(const ASTCheckQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - std::string nl_or_nothing = one_line ? "" : "\n"; - - std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); - std::string nl_or_ws = one_line ? " " : "\n"; - - s << (hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (hilite ? hilite_none : ""); - - if (!ast.table.empty()) - { - if (!ast.database.empty()) - { - s << (hilite ? hilite_keyword : "") << indent_str << ast.database << (hilite ? hilite_none : ""); - s << "."; - } - s << (hilite ? hilite_keyword : "") << indent_str << ast.table << (hilite ? hilite_none : ""); - } - s << nl_or_ws; -} - -/* -void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - s << (hilite ? hilite_keyword : "") << "{" << (hilite ? hilite_none : ""); - - for (const auto & child : ast.children) - { - s << "\n"; - formatAST(*child, s, indent + 1, hilite, one_line, need_parens); - s << ";\n"; - } - - s << (hilite ? hilite_keyword : "") << "}" << (hilite ? hilite_none : ""); -}*/ - - -void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) -{ - -#define DISPATCH(NAME) \ - else if (const AST ## NAME * concrete = typeid_cast(&ast)) \ - formatAST(*concrete, s, indent, hilite, one_line, need_parens); - - if (false) {} - DISPATCH(SelectQuery) - DISPATCH(InsertQuery) - DISPATCH(CreateQuery) - DISPATCH(DropQuery) - DISPATCH(RenameQuery) - DISPATCH(ShowTablesQuery) - DISPATCH(UseQuery) - DISPATCH(SetQuery) - DISPATCH(OptimizeQuery) - DISPATCH(ExistsQuery) - DISPATCH(ShowCreateQuery) - DISPATCH(DescribeQuery) - DISPATCH(ExpressionList) - DISPATCH(Function) - DISPATCH(Identifier) - DISPATCH(Literal) - DISPATCH(NameTypePair) - DISPATCH(ColumnDeclaration) - DISPATCH(Asterisk) - DISPATCH(OrderByElement) - DISPATCH(Subquery) - DISPATCH(AlterQuery) - DISPATCH(ShowProcesslistQuery) - DISPATCH(Set) - DISPATCH(Join) - DISPATCH(CheckQuery) -// DISPATCH(MultiQuery) - else - throw Exception("Unknown element in AST: " + ast.getID() - + ((ast.range.first && (ast.range.second > ast.range.first)) - ? " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'" - : ""), - ErrorCodes::UNKNOWN_ELEMENT_IN_AST); - -#undef DISPATCH -} - - String formatColumnsForCreateQuery(NamesAndTypesList & columns) { std::string res; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 3c2e42a5ccb..a2bc499db88 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -485,15 +485,12 @@ int Server::main(const std::vector & args) if (config().has("interserver_http_port")) { - String this_host; - if (config().has("interserver_http_host")) - { - this_host = config().getString("interserver_http_host"); - } - else + String this_host = config().getString("interserver_http_host", ""); + + if (this_host.empty()) { this_host = getFQDNOrHostName(); - LOG_DEBUG(log, "Configuration parameter 'interserver_http_host' doesn't exist. Will use '" + this_host + "' as replica host."); + LOG_DEBUG(log, "Configuration parameter 'interserver_http_host' doesn't exist or exists and empty. Will use '" + this_host + "' as replica host."); } String port_str = config().getString("interserver_http_port"); diff --git a/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp b/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp index cadd0778888..cb2a091d6d5 100644 --- a/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -103,7 +103,7 @@ size_t ActiveDataPartSet::size() const } -String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level) +String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, Int64 left_id, Int64 right_id, UInt64 level) { const auto & date_lut = DateLUT::instance(); @@ -133,7 +133,7 @@ String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, U bool ActiveDataPartSet::isPartDirectory(const String & dir_name, Poco::RegularExpression::MatchVec * out_matches) { Poco::RegularExpression::MatchVec matches; - static Poco::RegularExpression file_name_regexp("^(\\d{8})_(\\d{8})_(\\d+)_(\\d+)_(\\d+)"); + static Poco::RegularExpression file_name_regexp("^(\\d{8})_(\\d{8})_(-?\\d+)_(-?\\d+)_(\\d+)"); bool res = (file_name_regexp.match(dir_name, 0, matches) && 6 == matches.size()); if (out_matches) *out_matches = matches; @@ -157,12 +157,17 @@ void ActiveDataPartSet::parsePartName(const String & file_name, Part & part, con part.left_date = date_lut.YYYYMMDDToDayNum(parse(file_name.substr(matches[1].offset, matches[1].length))); part.right_date = date_lut.YYYYMMDDToDayNum(parse(file_name.substr(matches[2].offset, matches[2].length))); - part.left = parse(file_name.substr(matches[3].offset, matches[3].length)); - part.right = parse(file_name.substr(matches[4].offset, matches[4].length)); + part.left = parse(file_name.substr(matches[3].offset, matches[3].length)); + part.right = parse(file_name.substr(matches[4].offset, matches[4].length)); part.level = parse(file_name.substr(matches[5].offset, matches[5].length)); - part.left_month = date_lut.toFirstDayNumOfMonth(part.left_date); - part.right_month = date_lut.toFirstDayNumOfMonth(part.right_date); + DayNum_t left_month = date_lut.toFirstDayNumOfMonth(part.left_date); + DayNum_t right_month = date_lut.toFirstDayNumOfMonth(part.right_date); + + if (left_month != right_month) + throw Exception("Part name " + file_name + " contains different months", ErrorCodes::BAD_DATA_PART_NAME); + + part.month = left_month; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 11cb7d298d8..5f21b4e775e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -112,9 +112,9 @@ MergeTreeData::MergeTreeData( throw Exception("Primary key could be empty only for UnsortedMergeTree", ErrorCodes::BAD_ARGUMENTS); } -UInt64 MergeTreeData::getMaxDataPartIndex() +Int64 MergeTreeData::getMaxDataPartIndex() { - UInt64 max_part_id = 0; + Int64 max_part_id = 0; for (const auto & part : data_parts) max_part_id = std::max(max_part_id, part->right); @@ -264,9 +264,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) while (curr_jt != data_parts.end()) { /// Куски данных за разные месяцы рассматривать не будем - if ((*curr_jt)->left_month != (*curr_jt)->right_month - || (*curr_jt)->right_month != (*prev_jt)->left_month - || (*prev_jt)->left_month != (*prev_jt)->right_month) + if ((*curr_jt)->month != (*prev_jt)->month) { ++prev_jt; ++curr_jt; @@ -734,6 +732,7 @@ MergeTreeData::DataPartsVector MergeTreeData::renameTempPartAndReplace( bool obsolete = false; /// Покрыт ли part каким-нибудь куском. DataPartsVector res; + /// Куски, содержащиеся в part, идут в data_parts подряд, задевая место, куда вставился бы сам part. DataParts::iterator it = data_parts.lower_bound(part); /// Пойдем влево. @@ -841,7 +840,7 @@ void MergeTreeData::renameAndDetachPart(const DataPartPtr & part, const String & Strings restored; bool error = false; - UInt64 pos = part->left; + Int64 pos = part->left; if (it != all_data_parts.begin()) { @@ -934,13 +933,13 @@ size_t MergeTreeData::getMaxPartsCountForMonth() for (const auto & part : data_parts) { - if (part->left_month == cur_month) + if (part->month == cur_month) { ++cur_count; } else { - cur_month = part->left_month; + cur_month = part->month; cur_count = 1; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 6d47840c076..f69f51be70f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -110,13 +110,6 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa continue; } - /// Кусок в одном месяце. - if (first_part->left_month != first_part->right_month) - { - LOG_WARNING(log, "Part " << first_part->name << " spans more than one month"); - continue; - } - /// Самый длинный валидный отрезок, начинающийся здесь. size_t cur_longest_max = -1U; size_t cur_longest_min = -1U; @@ -128,8 +121,8 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa size_t cur_sum = first_part->size_in_bytes; int cur_len = 1; - DayNum_t month = first_part->left_month; - UInt64 cur_id = first_part->right; + DayNum_t month = first_part->month; + Int64 cur_id = first_part->right; /// Этот месяц кончился хотя бы день назад. bool is_old_month = now_day - now_month >= 1 && now_month > month; @@ -151,9 +144,8 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa const MergeTreeData::DataPartPtr & last_part = *jt; /// Кусок разрешено сливать с предыдущим, и в одном правильном месяце. - if (last_part->left_month != last_part->right_month || - last_part->left_month != month || - !can_merge(prev_part, last_part)) + if (last_part->month != month + || !can_merge(prev_part, last_part)) { break; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 11a7fa0cb5a..87e0952c6b9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -71,7 +71,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b return res; } -MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDateInterval & block_with_dates, UInt64 temp_index) +MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDateInterval & block_with_dates, Int64 temp_index) { Block & block = block_with_dates.block; UInt16 min_date = block_with_dates.min_date; @@ -79,6 +79,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa const auto & date_lut = DateLUT::instance(); + DayNum_t min_month = date_lut.toFirstDayNumOfMonth(DayNum_t(min_date)); + DayNum_t max_month = date_lut.toFirstDayNumOfMonth(DayNum_t(max_date)); + + if (min_month != max_month) + throw Exception("Logical error: part spans more than one month."); + size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity; String tmp_part_name = "tmp_" + ActiveDataPartSet::getPartName( @@ -100,8 +106,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa SortDescription sort_descr = data.getSortDescription(); /// Сортируем. + IColumn::Permutation * perm_ptr = nullptr; + IColumn::Permutation perm; if (data.mode != MergeTreeData::Unsorted) - stableSortBlock(block, sort_descr); + { + stableGetPermutation(block, sort_descr, perm); + perm_ptr = &perm; + } NamesAndTypesList columns = data.getColumnsList().filter(block.getColumnsList().getNames()); MergedBlockOutputStream out(data, part_tmp_path, columns, CompressionMethod::LZ4); @@ -109,7 +120,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa out.getIndex().reserve(part_size * sort_descr.size()); out.writePrefix(); - out.write(block); + out.writeWithPermutation(block, perm_ptr); MergeTreeData::DataPart::Checksums checksums = out.writeSuffixAndGetChecksums(); new_data_part->left_date = DayNum_t(min_date); @@ -119,8 +130,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa new_data_part->level = 0; new_data_part->size = part_size; new_data_part->modification_time = time(0); - new_data_part->left_month = date_lut.toFirstDayNumOfMonth(new_data_part->left_date); - new_data_part->right_month = date_lut.toFirstDayNumOfMonth(new_data_part->right_date); + new_data_part->month = min_month; new_data_part->columns = columns; new_data_part->checksums = checksums; new_data_part->index.swap(out.getIndex()); diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 8d98faf9540..754354a8fe3 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include #include @@ -177,6 +178,13 @@ StoragePtr StorageFactory::get( materialized_columns, alias_columns, column_defaults, attach, context.getSettings().max_compress_block_size); } + else if (name == "StripeLog") + { + return StorageStripeLog::create( + data_path, table_name, columns, + materialized_columns, alias_columns, column_defaults, + attach, context.getSettings().max_compress_block_size); + } else if (name == "Set") { return StorageSet::create( diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index ac1c80ef74a..38c590995ff 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -129,7 +129,7 @@ private: { Stream(const std::string & data_path, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain) + compressed(plain, CompressionMethod::LZ4, max_compress_block_size) { plain_offset = Poco::File(data_path).getSize(); } @@ -463,7 +463,7 @@ StorageLog::StorageLog( : IStorage{materialized_columns_, alias_columns_, column_defaults_}, path(path_), name(name_), columns(columns_), loaded_marks(false), max_compress_block_size(max_compress_block_size_), - file_checker(path + escapeForFileName(name) + '/' + "sizes.json", *this) + file_checker(path + escapeForFileName(name) + '/' + "sizes.json") { if (columns->empty()) throw Exception("Empty list of columns passed to StorageLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 2b0a872d7ce..49123d7017d 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -297,7 +297,7 @@ void StorageMergeTree::dropPartition(const Field & partition, bool detach, bool for (const auto & part : parts) { - if (!(part->left_month == part->right_month && part->left_month == month)) + if (part->month != month) continue; LOG_DEBUG(log, "Removing part " << part->name); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 67653e5f588..a143094161f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -26,7 +26,7 @@ namespace DB const auto ERROR_SLEEP_MS = 1000; const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000; -const auto RESERVED_BLOCK_NUMBERS = 200; +const Int64 RESERVED_BLOCK_NUMBERS = 200; StorageReplicatedMergeTree::StorageReplicatedMergeTree( @@ -1272,7 +1272,8 @@ void StorageReplicatedMergeTree::mergeSelectingThread() auto zookeeper = getZooKeeper(); /// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам. - for (UInt64 number = left->right + 1; number <= right->left - 1; ++number) /// Номера блоков больше нуля. + /// Номера до RESERVED_BLOCK_NUMBERS всегда не соответствуют никаким блокам. + for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, left->right + 1); number <= right->left - 1; ++number) { String path1 = zookeeper_path + "/block_numbers/" + month_name + "/block-" + padIndex(number); String path2 = zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number); @@ -1402,7 +1403,7 @@ void StorageReplicatedMergeTree::mergeSelectingThread() for (size_t i = 0; i + 1 < parts.size(); ++i) { /// Уберем больше не нужные отметки о несуществующих блоках. - for (UInt64 number = parts[i]->right + 1; number <= parts[i + 1]->left - 1; ++number) + for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, parts[i]->right + 1); number <= parts[i + 1]->left - 1; ++number) { zookeeper->tryRemove(zookeeper_path + "/block_numbers/" + month_name + "/block-" + padIndex(number)); zookeeper->tryRemove(zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number)); @@ -2241,7 +2242,7 @@ void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partiti for (const auto & part : parts) { - if (!(part->left_month == part->right_month && part->left_month == month)) + if (part->month != month) continue; LOG_DEBUG(log, "Removing unreplicated part " << part->name); @@ -2279,7 +2280,7 @@ void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, * NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм, * чтобы гарантировать этот инвариант. */ - UInt64 right; + Int64 right; { AbandonableLockInZooKeeper block_number_lock = allocateBlockNumber(month_name); @@ -2329,7 +2330,7 @@ void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unrep String partition; if (attach_part) - partition = field.getType() == Field::Types::UInt64 ? toString(field.get()) : field.safeGet(); + partition = field.safeGet(); else partition = MergeTreeData::getMonthName(field); @@ -2369,19 +2370,16 @@ void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unrep /// Выделим добавляемым кускам максимальные свободные номера, меньшие RESERVED_BLOCK_NUMBERS. /// NOTE: Проверка свободности номеров никак не синхронизируется. Выполнять несколько запросов ATTACH/DETACH/DROP одновременно нельзя. - UInt64 min_used_number = RESERVED_BLOCK_NUMBERS; + Int64 min_used_number = RESERVED_BLOCK_NUMBERS; + DayNum_t month = DateLUT::instance().makeDayNum(parse(partition.substr(0, 4)), parse(partition.substr(4, 2)), 0); { - /// TODO Это необходимо лишь в пределах одного месяца. auto existing_parts = data.getDataParts(); for (const auto & part : existing_parts) - min_used_number = std::min(min_used_number, part->left); + if (part->month == month) + min_used_number = std::min(min_used_number, part->left); } - if (parts.size() > min_used_number) - throw Exception("Not enough free small block numbers for attaching parts: " - + toString(parts.size()) + " needed, " + toString(min_used_number) + " available", ErrorCodes::NOT_ENOUGH_BLOCK_NUMBERS); - /// Добавим записи в лог. std::reverse(parts.begin(), parts.end()); std::list entries; diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp new file mode 100644 index 00000000000..3659b987fd7 --- /dev/null +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -0,0 +1,257 @@ +#include + +#include +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include + + +namespace DB +{ + +#define INDEX_BUFFER_SIZE 4096 + + +class StripeLogBlockInputStream : public IProfilingBlockInputStream +{ +public: + StripeLogBlockInputStream(const NameSet & column_names_, StorageStripeLog & storage_, size_t max_read_buffer_size_, + const Poco::SharedPtr & index_, + IndexForNativeFormat::Blocks::const_iterator index_begin_, + IndexForNativeFormat::Blocks::const_iterator index_end_) + : column_names(column_names_.begin(), column_names_.end()), storage(storage_), + index(index_), index_begin(index_begin_), index_end(index_end_), + data_in(storage.full_path() + "data.bin", 0, 0, max_read_buffer_size_), + block_in(data_in, 0, true, index_begin, index_end) + { + } + + String getName() const override { return "StripeLog"; } + + String getID() const override + { + std::stringstream s; + s << "StripeLog"; + for (const auto & name : column_names) + s << ", " << name; /// NOTE Отсутствует эскейпинг. + return s.str(); + } + +protected: + Block readImpl() override + { + return block_in.read(); + } + +private: + NameSet column_names; + StorageStripeLog & storage; + + const Poco::SharedPtr index; + IndexForNativeFormat::Blocks::const_iterator index_begin; + IndexForNativeFormat::Blocks::const_iterator index_end; + + CompressedReadBufferFromFile data_in; + NativeBlockInputStream block_in; +}; + + +class StripeLogBlockOutputStream : public IBlockOutputStream +{ +public: + StripeLogBlockOutputStream(StorageStripeLog & storage_) + : storage(storage_), lock(storage.rwlock), + data_out_compressed(storage.full_path() + "data.bin"), + data_out(data_out_compressed, CompressionMethod::LZ4, storage.max_compress_block_size), + index_out_compressed(storage.full_path() + "index.mrk", INDEX_BUFFER_SIZE), + index_out(index_out_compressed), + block_out(data_out, 0, &index_out) + { + } + + ~StripeLogBlockOutputStream() + { + try + { + writeSuffix(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + void write(const Block & block) override + { + block_out.write(block); + } + + void writeSuffix() override + { + if (done) + return; + + block_out.writeSuffix(); + data_out.next(); + data_out_compressed.next(); + index_out.next(); + index_out_compressed.next(); + + FileChecker::Files files{ data_out_compressed.getFileName(), index_out_compressed.getFileName() }; + storage.file_checker.update(files.begin(), files.end()); + + done = true; + } + +private: + StorageStripeLog & storage; + Poco::ScopedWriteRWLock lock; + + WriteBufferFromFile data_out_compressed; + CompressedWriteBuffer data_out; + WriteBufferFromFile index_out_compressed; + CompressedWriteBuffer index_out; + NativeBlockOutputStream block_out; + + bool done = false; +}; + + +StorageStripeLog::StorageStripeLog( + const std::string & path_, + const std::string & name_, + NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + bool attach, + size_t max_compress_block_size_) + : IStorage{materialized_columns_, alias_columns_, column_defaults_}, + path(path_), name(name_), columns(columns_), + max_compress_block_size(max_compress_block_size_), + file_checker(path + escapeForFileName(name) + '/' + "sizes.json"), + log(&Logger::get("StorageStripeLog")) +{ + if (columns->empty()) + throw Exception("Empty list of columns passed to StorageStripeLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); + + String full_path = path + escapeForFileName(name) + '/'; + if (!attach) + { + /// создаём файлы, если их нет + if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST) + throwFromErrno("Cannot create directory " + full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY); + } +} + +StoragePtr StorageStripeLog::create( + const std::string & path_, + const std::string & name_, + NamesAndTypesListPtr columns_, + const NamesAndTypesList & materialized_columns_, + const NamesAndTypesList & alias_columns_, + const ColumnDefaults & column_defaults_, + bool attach, + size_t max_compress_block_size_) +{ + return (new StorageStripeLog{ + path_, name_, columns_, + materialized_columns_, alias_columns_, column_defaults_, + attach, max_compress_block_size_ + })->thisPtr(); +} + + +void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) +{ + Poco::ScopedWriteRWLock lock(rwlock); + + /// Переименовываем директорию с данными. + Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_table_name)); + + path = new_path_to_db; + name = new_table_name; + file_checker.setPath(path + escapeForFileName(name) + "/" + "sizes.json"); +} + + +BlockInputStreams StorageStripeLog::read( + const Names & column_names, + ASTPtr query, + const Context & context, + const Settings & settings, + QueryProcessingStage::Enum & processed_stage, + const size_t max_block_size, + unsigned threads) +{ + Poco::ScopedReadRWLock lock(rwlock); + + check(column_names); + processed_stage = QueryProcessingStage::FetchColumns; + + NameSet column_names_set(column_names.begin(), column_names.end()); + + CompressedReadBufferFromFile index_in(full_path() + "index.mrk", 0, 0, INDEX_BUFFER_SIZE); + Poco::SharedPtr index = new IndexForNativeFormat(index_in, column_names_set); + + BlockInputStreams res; + + size_t size = index->blocks.size(); + if (threads > size) + threads = size; + + for (size_t thread = 0; thread < threads; ++thread) + { + IndexForNativeFormat::Blocks::const_iterator begin = index->blocks.begin(); + IndexForNativeFormat::Blocks::const_iterator end = index->blocks.begin(); + + std::advance(begin, thread * size / threads); + std::advance(end, (thread + 1) * size / threads); + + res.emplace_back(new StripeLogBlockInputStream(column_names_set, *this, settings.max_read_buffer_size, index, begin, end)); + } + + /// Непосредственно во время чтения не держим read lock, потому что мы читаем диапазоны данных, которые не меняются. + + return res; +} + + +BlockOutputStreamPtr StorageStripeLog::write( + ASTPtr query) +{ + return new StripeLogBlockOutputStream(*this); +} + + +bool StorageStripeLog::checkData() const +{ + Poco::ScopedReadRWLock lock(const_cast(rwlock)); + return file_checker.check(); +} + +} diff --git a/dbms/src/Storages/StorageSystemColumns.cpp b/dbms/src/Storages/StorageSystemColumns.cpp index 83ced0d22f4..b87872eec1a 100644 --- a/dbms/src/Storages/StorageSystemColumns.cpp +++ b/dbms/src/Storages/StorageSystemColumns.cpp @@ -1,6 +1,10 @@ #include +#include +#include +#include #include #include +#include #include #include @@ -15,7 +19,8 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "name", new DataTypeString }, { "type", new DataTypeString }, { "default_type", new DataTypeString }, - { "default_expression", new DataTypeString } + { "default_expression", new DataTypeString }, + { "bytes", new DataTypeUInt64 }, } { } @@ -103,6 +108,7 @@ BlockInputStreams StorageSystemColumns::read( ColumnPtr type_column = new ColumnString; ColumnPtr default_type_column = new ColumnString; ColumnPtr default_expression_column = new ColumnString; + ColumnPtr bytes_column = new ColumnUInt64; size_t rows = filtered_database_column->size(); for (size_t i = 0; i < rows; ++i) @@ -112,6 +118,7 @@ BlockInputStreams StorageSystemColumns::read( NamesAndTypesList columns; ColumnDefaults column_defaults; + std::unordered_map column_sizes; { StoragePtr storage = storages.at(std::make_pair(database_name, table_name)); @@ -120,6 +127,26 @@ BlockInputStreams StorageSystemColumns::read( columns = storage->getColumnsList(); columns.insert(std::end(columns), std::begin(storage->alias_columns), std::end(storage->alias_columns)); column_defaults = storage->column_defaults; + + /** Данные о размерах столбцов для таблиц семейства MergeTree. + * NOTE: В дальнейшем можно сделать интерфейс, позволяющий получить размеры столбцов у IStorage. + */ + if (auto storage_concrete = dynamic_cast(storage.get())) + { + column_sizes = storage_concrete->getData().getColumnSizes(); + } + else if (auto storage_concrete = dynamic_cast(storage.get())) + { + column_sizes = storage_concrete->getData().getColumnSizes(); + + auto unreplicated_data = storage_concrete->getUnreplicatedData(); + if (unreplicated_data) + { + auto unreplicated_column_sizes = unreplicated_data->getColumnSizes(); + for (const auto & name_size : unreplicated_column_sizes) + column_sizes[name_size.first] += name_size.second; + } + } } for (const auto & column : columns) @@ -129,16 +156,26 @@ BlockInputStreams StorageSystemColumns::read( name_column->insert(column.name); type_column->insert(column.type->getName()); - const auto it = column_defaults.find(column.name); - if (it == std::end(column_defaults)) { - default_type_column->insertDefault(); - default_expression_column->insertDefault(); + const auto it = column_defaults.find(column.name); + if (it == std::end(column_defaults)) + { + default_type_column->insertDefault(); + default_expression_column->insertDefault(); + } + else + { + default_type_column->insert(toString(it->second.type)); + default_expression_column->insert(queryToString(it->second.expression)); + } } - else + { - default_type_column->insert(toString(it->second.type)); - default_expression_column->insert(queryToString(it->second.expression)); + const auto it = column_sizes.find(column.name); + if (it == std::end(column_sizes)) + bytes_column->insertDefault(); + else + bytes_column->insert(it->second); } } } @@ -151,6 +188,7 @@ BlockInputStreams StorageSystemColumns::read( block.insert(ColumnWithTypeAndName(type_column, new DataTypeString, "type")); block.insert(ColumnWithTypeAndName(default_type_column, new DataTypeString, "default_type")); block.insert(ColumnWithTypeAndName(default_expression_column, new DataTypeString, "default_expression")); + block.insert(ColumnWithTypeAndName(bytes_column, new DataTypeUInt64, "bytes")); return BlockInputStreams{ 1, new OneBlockInputStream(block) }; } diff --git a/dbms/src/Storages/StorageSystemParts.cpp b/dbms/src/Storages/StorageSystemParts.cpp index 67f6170dc0b..558f6a90564 100644 --- a/dbms/src/Storages/StorageSystemParts.cpp +++ b/dbms/src/Storages/StorageSystemParts.cpp @@ -29,8 +29,8 @@ StorageSystemParts::StorageSystemParts(const std::string & name_) {"refcount", new DataTypeUInt32}, {"min_date", new DataTypeDate}, {"max_date", new DataTypeDate}, - {"min_block_number", new DataTypeUInt64}, - {"max_block_number", new DataTypeUInt64}, + {"min_block_number", new DataTypeInt64}, + {"max_block_number", new DataTypeInt64}, {"level", new DataTypeUInt32}, {"database", new DataTypeString}, @@ -169,8 +169,8 @@ BlockInputStreams StorageSystemParts::read( ColumnPtr refcount_column = new ColumnUInt32; ColumnPtr min_date_column = new ColumnUInt16; ColumnPtr max_date_column = new ColumnUInt16; - ColumnPtr min_block_number_column = new ColumnUInt64; - ColumnPtr max_block_number_column = new ColumnUInt64; + ColumnPtr min_block_number_column = new ColumnInt64; + ColumnPtr max_block_number_column = new ColumnInt64; ColumnPtr level_column = new ColumnUInt32; for (size_t i = 0; i < filtered_database_column->size();) @@ -227,7 +227,7 @@ BlockInputStreams StorageSystemParts::read( table_column->insert(table); engine_column->insert(engine); - mysqlxx::Date partition_date {part->left_month}; + mysqlxx::Date partition_date {part->month}; String partition = toString(partition_date.year()) + (partition_date.month() < 10 ? "0" : "") + toString(partition_date.month()); partition_column->insert(partition); @@ -263,8 +263,8 @@ BlockInputStreams StorageSystemParts::read( block.insert(ColumnWithTypeAndName(refcount_column, new DataTypeUInt32, "refcount")); block.insert(ColumnWithTypeAndName(min_date_column, new DataTypeDate, "min_date")); block.insert(ColumnWithTypeAndName(max_date_column, new DataTypeDate, "max_date")); - block.insert(ColumnWithTypeAndName(min_block_number_column, new DataTypeUInt64, "min_block_number")); - block.insert(ColumnWithTypeAndName(max_block_number_column, new DataTypeUInt64, "max_block_number")); + block.insert(ColumnWithTypeAndName(min_block_number_column, new DataTypeInt64, "min_block_number")); + block.insert(ColumnWithTypeAndName(max_block_number_column, new DataTypeInt64, "max_block_number")); block.insert(ColumnWithTypeAndName(level_column, new DataTypeUInt32, "level")); block.insert(ColumnWithTypeAndName(database_column, new DataTypeString, "database")); block.insert(ColumnWithTypeAndName(table_column, new DataTypeString, "table")); diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 89bc91d6084..2214e8e1f7f 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -108,7 +108,7 @@ private: { Stream(const std::string & data_path, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain) + compressed(plain, CompressionMethod::LZ4, max_compress_block_size) { } @@ -352,7 +352,7 @@ StorageTinyLog::StorageTinyLog( : IStorage{materialized_columns_, alias_columns_, column_defaults_}, path(path_), name(name_), columns(columns_), max_compress_block_size(max_compress_block_size_), - file_checker(path + escapeForFileName(name) + '/' + "sizes.json", *this), + file_checker(path + escapeForFileName(name) + '/' + "sizes.json"), log(&Logger::get("StorageTinyLog")) { if (columns->empty()) @@ -467,9 +467,4 @@ bool StorageTinyLog::checkData() const return file_checker.check(); } -StorageTinyLog::Files_t & StorageTinyLog::getFiles() -{ - return files; -} - } diff --git a/dbms/tests/queries/0_stateless/00204_extract_url_parameter.reference b/dbms/tests/queries/0_stateless/00204_extract_url_parameter.reference new file mode 100644 index 00000000000..58c9bdf9d01 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00204_extract_url_parameter.reference @@ -0,0 +1 @@ +111 diff --git a/dbms/tests/queries/0_stateless/00204_extract_url_parameter.sql b/dbms/tests/queries/0_stateless/00204_extract_url_parameter.sql new file mode 100644 index 00000000000..d6ca5b31333 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00204_extract_url_parameter.sql @@ -0,0 +1 @@ +SELECT extractURLParameter('http://test.com/?testq=aaa&q=111', 'q'); diff --git a/dbms/tests/queries/0_stateless/00205_scalar_subqueries.reference b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.reference new file mode 100644 index 00000000000..7b3ebbc7519 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.reference @@ -0,0 +1,5 @@ +1 1 +1 +1 1 +('2015-01-02','Hello') +('2015-01-02','Hello') ('2015-01-02','Hello') 1 1 diff --git a/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql new file mode 100644 index 00000000000..f924ff291ea --- /dev/null +++ b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql @@ -0,0 +1,5 @@ +SELECT (SELECT (SELECT (SELECT (SELECT (SELECT count() FROM (SELECT * FROM system.numbers LIMIT 10)))))) = (SELECT 10), ((SELECT 1, 'Hello', [1, 2]).3)[1]; +SELECT toUInt64((SELECT 9)) IN (SELECT number FROM system.numbers LIMIT 10); +SELECT (SELECT toDate('2015-01-02')) = toDate('2015-01-02'), 'Hello' = (SELECT 'Hello'); +SELECT (SELECT toDate('2015-01-02'), 'Hello'); +SELECT (SELECT toDate('2015-01-02'), 'Hello') AS x, x, identity((SELECT 1)), identity((SELECT 1) AS y); diff --git a/dbms/tests/queries/0_stateless/00206_empty_array_to_single.reference b/dbms/tests/queries/0_stateless/00206_empty_array_to_single.reference new file mode 100644 index 00000000000..e4e7e38fa36 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00206_empty_array_to_single.reference @@ -0,0 +1,14 @@ +[1,2] +[0] +[4,5,6] +[''] ['0000-00-00'] ['0000-00-00 00:00:00'] +[0] [''] ['0000-00-00 00:00:00'] ['0000-00-00'] +[0] ['0'] ['2015-01-01 00:00:00'] ['2015-01-01'] +[0,1] [''] ['2015-01-01 00:00:00','2015-01-01 00:00:01'] ['2015-01-01','2015-01-02'] +[0] ['0'] ['2015-01-01 00:00:00','2015-01-01 00:00:01','2015-01-01 00:00:02'] ['2015-01-01','2015-01-02','2015-01-03'] +[0] [''] ['2015-01-01 00:00:00','2015-01-01 00:00:01','2015-01-01 00:00:02','2015-01-01 00:00:03'] ['0000-00-00'] +[0,1] ['0'] ['0000-00-00 00:00:00'] ['2015-01-01'] +[0] [''] ['2015-01-01 00:00:00'] ['2015-01-01','2015-01-02'] +[0] ['0'] ['2015-01-01 00:00:00','2015-01-01 00:00:01'] ['2015-01-01','2015-01-02','2015-01-03'] +[0,1] [''] ['2015-01-01 00:00:00','2015-01-01 00:00:01','2015-01-01 00:00:02'] ['0000-00-00'] +[0] ['0'] ['2015-01-01 00:00:00','2015-01-01 00:00:01','2015-01-01 00:00:02','2015-01-01 00:00:03'] ['2015-01-01'] diff --git a/dbms/tests/queries/0_stateless/00206_empty_array_to_single.sql b/dbms/tests/queries/0_stateless/00206_empty_array_to_single.sql new file mode 100644 index 00000000000..0ad2975fa7f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00206_empty_array_to_single.sql @@ -0,0 +1,8 @@ +SELECT emptyArrayToSingle(arrayFilter(x -> x != 99, arrayJoin([[1, 2], [99], [4, 5, 6]]))); +SELECT emptyArrayToSingle(emptyArrayString()), emptyArrayToSingle(emptyArrayDate()), emptyArrayToSingle(emptyArrayDateTime()); + +SELECT + emptyArrayToSingle(range(number % 3)), + emptyArrayToSingle(arrayMap(x -> toString(x), range(number % 2))), + emptyArrayToSingle(arrayMap(x -> toDateTime('2015-01-01 00:00:00') + x, range(number % 5))), + emptyArrayToSingle(arrayMap(x -> toDate('2015-01-01') + x, range(number % 4))) FROM system.numbers LIMIT 10; diff --git a/dbms/tests/queries/0_stateless/00207_left_array_join.reference b/dbms/tests/queries/0_stateless/00207_left_array_join.reference new file mode 100644 index 00000000000..10ec6a7a16f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00207_left_array_join.reference @@ -0,0 +1,23 @@ +0 +1 +2 +2 +3 +4 +5 +5 +6 +7 +0 [] 0 +1 [0] 0 +2 [0,1] 0 +2 [0,1] 1 +3 [] 0 +4 [0] 0 +5 [0,1] 0 +5 [0,1] 1 +6 [] 0 +7 [0] 0 +8 [0,1] 0 +8 [0,1] 1 +9 [] 0 diff --git a/dbms/tests/queries/0_stateless/00207_left_array_join.sql b/dbms/tests/queries/0_stateless/00207_left_array_join.sql new file mode 100644 index 00000000000..8186054c250 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00207_left_array_join.sql @@ -0,0 +1,2 @@ +SELECT number FROM system.numbers LEFT ARRAY JOIN range(number % 3) AS arr LIMIT 10; +SELECT number, arr, x FROM (SELECT number, range(number % 3) AS arr FROM system.numbers LIMIT 10) LEFT ARRAY JOIN arr AS x; diff --git a/dbms/tests/queries/0_stateless/00208_agg_state_merge.reference b/dbms/tests/queries/0_stateless/00208_agg_state_merge.reference new file mode 100644 index 00000000000..d21a7aa01e4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00208_agg_state_merge.reference @@ -0,0 +1,7 @@ +0 15 15 +1 14 14 +2 14 14 +3 15 15 +4 9 9 +5 9 9 +6 9 9 diff --git a/dbms/tests/queries/0_stateless/00208_agg_state_merge.sql b/dbms/tests/queries/0_stateless/00208_agg_state_merge.sql new file mode 100644 index 00000000000..3f30f66dd44 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00208_agg_state_merge.sql @@ -0,0 +1 @@ +SELECT k % 7 AS k2, finalizeAggregation(uniqMergeState(state)), uniqMerge(state) FROM (SELECT k, uniqState(x) AS state FROM (SELECT number % 11 AS k, intDiv(number, 7) AS x FROM system.numbers LIMIT 100) GROUP BY k) GROUP BY k2 ORDER BY k2; diff --git a/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference b/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference new file mode 100644 index 00000000000..e86726625a1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00209_insert_select_extremes.reference @@ -0,0 +1,11 @@ + +1 +1 + +0 + +0 + +1 +1 +4 1 1 diff --git a/dbms/tests/queries/0_stateless/00209_insert_select_extremes.sql b/dbms/tests/queries/0_stateless/00209_insert_select_extremes.sql new file mode 100644 index 00000000000..0d632992b67 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00209_insert_select_extremes.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS test.test; +CREATE TABLE test.test (x UInt8) ENGINE = Log; + +INSERT INTO test.test SELECT 1 AS x; +INSERT INTO test.test SELECT 1 AS x SETTINGS extremes = 1; +INSERT INTO test.test SELECT 1 AS x GROUP BY 1 WITH TOTALS; +INSERT INTO test.test SELECT 1 AS x GROUP BY 1 WITH TOTALS SETTINGS extremes = 1; + +SELECT count(), min(x), max(x) FROM test.test; + +DROP TABLE test.test; diff --git a/dbms/tests/queries/0_stateless/00210_insert_select_extremes_http.reference b/dbms/tests/queries/0_stateless/00210_insert_select_extremes_http.reference new file mode 100644 index 00000000000..016f3290af0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00210_insert_select_extremes_http.reference @@ -0,0 +1,3 @@ + +1 +1 diff --git a/dbms/tests/queries/0_stateless/00210_insert_select_extremes_http.sh b/dbms/tests/queries/0_stateless/00210_insert_select_extremes_http.sh new file mode 100755 index 00000000000..e9b82bccfa5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00210_insert_select_extremes_http.sh @@ -0,0 +1,6 @@ +#!/bin/bash + +curl -sS http://localhost:8123/?extremes=1 -d @- <<< "DROP TABLE IF EXISTS test.test" +curl -sS http://localhost:8123/?extremes=1 -d @- <<< "CREATE TABLE test.test (x UInt8) ENGINE = Log" +curl -sS http://localhost:8123/?extremes=1 -d @- <<< "INSERT INTO test.test SELECT 1 AS x" +curl -sS http://localhost:8123/?extremes=1 -d @- <<< "DROP TABLE test.test" diff --git a/dbms/tests/queries/0_stateless/00211_query_formatting_aliases.reference b/dbms/tests/queries/0_stateless/00211_query_formatting_aliases.reference new file mode 100644 index 00000000000..b1cd860dcc6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00211_query_formatting_aliases.reference @@ -0,0 +1,3 @@ +0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 +1 1 (2,2) +1 1 (2,2) diff --git a/dbms/tests/queries/0_stateless/00211_query_formatting_aliases.sql b/dbms/tests/queries/0_stateless/00211_query_formatting_aliases.sql new file mode 100644 index 00000000000..e1006d89d82 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00211_query_formatting_aliases.sql @@ -0,0 +1,8 @@ +SELECT toUInt64(1) IN (1234567890, 2345678901, 3456789012, 4567890123, 5678901234, 6789012345, 7890123456, 8901234567, 9012345678, 123456789) AS x, + x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, + x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, + x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, + x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x +FROM remote('127.0.0.1', system, one) SETTINGS max_query_size = 10000; + +SELECT 1 AS x, x, (SELECT 2 AS x, x) FROM remote('127.0.0.{1,2}', system.one) WHERE (3, 4) IN (SELECT 3 AS x, toUInt8(x + 1)); diff --git a/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference b/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference new file mode 100644 index 00000000000..d66effa9fb1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference @@ -0,0 +1,416 @@ +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 159 +1 164 +3 165 +6 162 +7 160 +9 164 +10 81 +11 158 +13 161 +14 160 +17 163 +19 164 +20 159 +21 161 +22 159 +26 160 +31 164 +35 160 +36 161 +0 54571 +1 55013 +3 52912 +6 52353 +7 54011 +9 54138 +10 26870 +11 54554 +13 53951 +14 53396 +17 55227 +19 55115 +20 54370 +21 54268 +22 54620 +26 53394 +31 54151 +35 54328 +36 52997 +0.125 1 +0.5 1 +0.05 1 +0.143 1 +0.056 1 +0.048 2 +0.083 1 +0.25 1 +0.1 1 +0.028 1 +0.027 1 +0.031 1 +0.067 1 +0.037 1 +0.045 161 +0.125 160 +0.5 164 +0.05 164 +0.143 162 +0.091 81 +0.056 163 +0.048 159 +0.083 158 +0.25 165 +1 159 +0.1 164 +0.028 160 +0.027 161 +0.031 164 +0.067 160 +0.043 159 +0.037 160 +0.071 161 +0.045 54268 +0.125 54011 +0.5 55013 +0.05 55115 +0.143 52353 +0.091 26870 +0.056 55227 +0.048 54370 +0.083 54554 +0.25 52912 +1 54571 +0.1 54138 +0.028 54328 +0.027 52997 +0.031 54151 +0.067 53396 +0.043 54620 +0.037 53394 +0.071 53951 +0.5 1 +0.05 1 +0.25 1 +0.048 2 +0.083 1 +0.125 1 +0.031 1 +0.143 1 +0.028 1 +0.067 1 +0.027 1 +0.056 1 +0.037 1 +0.1 1 +0.5 164 +0.05 164 +0.25 165 +0.048 159 +0.091 81 +0.043 159 +0.071 161 +0.083 158 +0.125 160 +0.031 164 +0.143 162 +0.028 160 +0.067 160 +0.045 161 +0.027 161 +0.056 163 +0.037 160 +0.1 164 +1 159 +0.5 55013 +0.05 55115 +0.25 52912 +0.048 54370 +0.091 26870 +0.043 54620 +0.071 53951 +0.083 54554 +0.125 54011 +0.031 54151 +0.143 52353 +0.028 54328 +0.067 53396 +0.045 54268 +0.027 52997 +0.056 55227 +0.037 53394 +0.1 54138 +1 54571 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 158 +3 162 +6 163 +7 162 +9 162 +10 79 +11 162 +13 163 +14 160 +17 163 +19 158 +20 162 +21 157 +22 164 +26 162 +31 161 +35 162 +36 163 +0 54029 +1 53772 +3 53540 +6 54012 +7 53910 +9 52761 +10 26462 +11 52701 +13 54505 +14 53790 +17 54064 +19 55420 +20 56686 +21 52639 +22 54251 +26 53827 +31 53574 +35 55022 +36 53961 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 +0 54226 +1 54034 +3 54016 +6 53982 +7 54076 +9 54218 +10 27075 +11 54093 +13 54108 +14 54096 +17 54294 +19 54070 +20 54028 +21 54170 +22 54106 +26 54103 +31 54050 +35 54130 +36 53868 +0.125 1 +0.5 1 +0.05 1 +0.143 1 +0.056 1 +0.048 2 +0.083 1 +0.25 1 +0.1 1 +0.028 1 +0.027 1 +0.031 1 +0.067 1 +0.037 1 +0.045 162 +0.125 163 +0.5 162 +0.05 162 +0.143 162 +0.091 81 +0.056 162 +0.048 162 +0.083 163 +0.25 162 +1 162 +0.1 163 +0.028 162 +0.027 162 +0.031 162 +0.067 162 +0.043 162 +0.037 162 +0.071 162 +0.045 54170 +0.125 54076 +0.5 54034 +0.05 54070 +0.143 53982 +0.091 27075 +0.056 54294 +0.048 54028 +0.083 54093 +0.25 54016 +1 54226 +0.1 54218 +0.028 54130 +0.027 53868 +0.031 54050 +0.067 54096 +0.043 54106 +0.037 54103 +0.071 54108 +0.5 1 +0.05 1 +0.25 1 +0.048 2 +0.083 1 +0.125 1 +0.031 1 +0.143 1 +0.028 1 +0.067 1 +0.027 1 +0.056 1 +0.037 1 +0.1 1 +0.5 162 +0.05 162 +0.25 162 +0.048 162 +0.091 81 +0.043 162 +0.071 162 +0.083 163 +0.125 163 +0.031 162 +0.143 162 +0.028 162 +0.067 162 +0.045 162 +0.027 162 +0.056 162 +0.037 162 +0.1 163 +1 162 +0.5 54034 +0.05 54070 +0.25 54016 +0.048 54028 +0.091 27075 +0.043 54106 +0.071 54108 +0.083 54093 +0.125 54076 +0.031 54050 +0.143 53982 +0.028 54130 +0.067 54096 +0.045 54170 +0.027 53868 +0.056 54294 +0.037 54103 +0.1 54218 +1 54226 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 +0 54195 +1 54086 +3 54127 +6 54173 +7 53969 +9 54257 +10 26985 +11 53880 +13 54105 +14 54043 +17 54176 +19 53913 +20 54088 +21 53991 +22 54112 +26 54136 +31 54074 +35 54153 +36 53999 diff --git a/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.sql b/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.sql new file mode 100644 index 00000000000..2886daeb3b3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.sql @@ -0,0 +1,35 @@ +/* uniqHLL12 */ + +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +/* uniqCombined */ + +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + +SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; diff --git a/dbms/tests/queries/0_stateless/00213_multiple_global_in.reference b/dbms/tests/queries/0_stateless/00213_multiple_global_in.reference new file mode 100644 index 00000000000..9972842f982 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00213_multiple_global_in.reference @@ -0,0 +1 @@ +1 1 diff --git a/dbms/tests/queries/0_stateless/00213_multiple_global_in.sql b/dbms/tests/queries/0_stateless/00213_multiple_global_in.sql new file mode 100644 index 00000000000..b93c2bec722 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00213_multiple_global_in.sql @@ -0,0 +1 @@ +SELECT 1 GLOBAL IN (SELECT 1), 2 GLOBAL IN (SELECT 2) FROM remote('127.0.0.2', system.one); diff --git a/dbms/tests/queries/0_stateless/00214_primary_key_order.reference b/dbms/tests/queries/0_stateless/00214_primary_key_order.reference new file mode 100644 index 00000000000..f9a71b1af97 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00214_primary_key_order.reference @@ -0,0 +1,9 @@ +1 +2 +3 +b -3 +c -3 +c -2 +d -3 +d -2 +d -1 diff --git a/dbms/tests/queries/0_stateless/00214_primary_key_order.sql b/dbms/tests/queries/0_stateless/00214_primary_key_order.sql new file mode 100644 index 00000000000..b2c00fb1f63 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00214_primary_key_order.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test.primary_key; +CREATE TABLE test.primary_key (d Date DEFAULT today(), x Int8) ENGINE = MergeTree(d, -x, 1); + +INSERT INTO test.primary_key (x) VALUES (1), (2), (3); + +SELECT x FROM test.primary_key ORDER BY x; + +SELECT 'a', -x FROM test.primary_key WHERE -x < -3; +SELECT 'b', -x FROM test.primary_key WHERE -x < -2; +SELECT 'c', -x FROM test.primary_key WHERE -x < -1; +SELECT 'd', -x FROM test.primary_key WHERE -x < toInt8(0); + +DROP TABLE test.primary_key; diff --git a/dbms/tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference b/dbms/tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference new file mode 100644 index 00000000000..7f43a43f889 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00215_primary_key_order_zookeeper.reference @@ -0,0 +1,5 @@ +1 +2 +3 +2 +3 diff --git a/dbms/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql b/dbms/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql new file mode 100644 index 00000000000..cd86bc7aa15 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00215_primary_key_order_zookeeper.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test.primary_key; +CREATE TABLE test.primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/primary_key', 'r1', d, -x, 1); + +INSERT INTO test.primary_key (x) VALUES (1), (2), (3); +INSERT INTO test.primary_key (x) VALUES (1), (3), (2); +INSERT INTO test.primary_key (x) VALUES (2), (1), (3); +INSERT INTO test.primary_key (x) VALUES (2), (3), (1); +INSERT INTO test.primary_key (x) VALUES (3), (1), (2); +INSERT INTO test.primary_key (x) VALUES (3), (2), (1); + +SELECT x FROM test.primary_key ORDER BY x; +SELECT x FROM test.primary_key WHERE -x < -1 ORDER BY x; + +DROP TABLE test.primary_key; diff --git a/libs/libmysqlxx/src/PoolWithFailover.cpp b/libs/libmysqlxx/src/PoolWithFailover.cpp index c98b42036d2..5fe4c64dfdf 100644 --- a/libs/libmysqlxx/src/PoolWithFailover.cpp +++ b/libs/libmysqlxx/src/PoolWithFailover.cpp @@ -13,10 +13,8 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & cfg cfg.keys(config_name, replica_keys); for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = replica_keys.begin(); it != replica_keys.end(); ++it) { - if (!(*it == "port" || *it == "user" || *it == "password" || *it == "db" || *it == "table")) + if (*it == "replica") /// На том же уровне могут быть другие параметры. { - if (it->size() < std::string("replica").size() || it->substr(0, std::string("replica").size()) != "replica") - throw Poco::Exception("Unknown element in config: " + *it + ", expected replica"); std::string replica_name = config_name + "." + *it; Replica replica(new Pool(cfg, replica_name, default_connections, max_connections, config_name.c_str()), cfg.getInt(replica_name + ".priority", 0));