diff --git a/dbms/benchmark/vertica/README b/dbms/benchmark/vertica/README index 34525e46597..91af3e41e27 100644 --- a/dbms/benchmark/vertica/README +++ b/dbms/benchmark/vertica/README @@ -8,6 +8,7 @@ Download HP Vertica 7.1.1 Analytic Database Server, Debian or Ubuntu 14.04 versi sudo apt-get install sysstat pstack mcelog sudo dpkg -i vertica_7.1.1-0_amd64.deb sudo sh -c "echo 'export TZ=Europe/Moscow' >> /home/dbadmin/.bash_profile" +# Don't specify localhost due to poor support of IPv6. sudo /opt/vertica/sbin/install_vertica --hosts=127.0.0.1 --failure-threshold=NONE sudo mkdir /opt/vertica-data/ sudo chown dbadmin /opt/vertica-data/ @@ -23,8 +24,6 @@ both directories: /opt/vertica-data/ main menu exit -PS. Note that Vertica doesn't support IPv6. - How to prepare data ------------------- diff --git a/dbms/include/DB/Client/ConnectionPool.h b/dbms/include/DB/Client/ConnectionPool.h index 3e65e149a74..c4dd3d69a60 100644 --- a/dbms/include/DB/Client/ConnectionPool.h +++ b/dbms/include/DB/Client/ConnectionPool.h @@ -26,13 +26,13 @@ class IConnectionPool : private boost::noncopyable { public: typedef PoolBase::Entry Entry; - virtual Entry get(Settings * settings = nullptr) = 0; + virtual Entry get(const Settings * settings = nullptr) = 0; /** Выделяет до указанного количества соединений для работы. * Соединения предоставляют доступ к разным репликам одного шарда. * Выкидывает исключение, если не удалось выделить ни одного соединения. */ - virtual std::vector getMany(Settings * settings = nullptr) + virtual std::vector getMany(const Settings * settings = nullptr) { return std::vector{ get(settings) }; } @@ -89,7 +89,7 @@ public: /** Выделяет соединение для работы. */ - Entry get(Settings * settings = nullptr) override + Entry get(const Settings * settings = nullptr) override { if (settings) return Base::get(settings->queue_max_wait_ms.totalMilliseconds()); diff --git a/dbms/include/DB/Client/ConnectionPoolWithFailover.h b/dbms/include/DB/Client/ConnectionPoolWithFailover.h index bded7ff3614..e2f42366341 100644 --- a/dbms/include/DB/Client/ConnectionPoolWithFailover.h +++ b/dbms/include/DB/Client/ConnectionPoolWithFailover.h @@ -21,11 +21,11 @@ namespace DB * * Замечание: если один из вложенных пулов заблокируется из-за переполнения, то этот пул тоже заблокируется. */ -class ConnectionPoolWithFailover : public PoolWithFailoverBase, public IConnectionPool +class ConnectionPoolWithFailover : public PoolWithFailoverBase, public IConnectionPool { public: typedef IConnectionPool::Entry Entry; - typedef PoolWithFailoverBase Base; + typedef PoolWithFailoverBase Base; ConnectionPoolWithFailover(ConnectionPools & nested_pools_, LoadBalancing load_balancing, @@ -52,7 +52,7 @@ public: } /** Выделяет соединение для работы. */ - Entry get(Settings * settings = nullptr) override + Entry get(const Settings * settings = nullptr) override { applyLoadBalancing(settings); return Base::get(settings); @@ -61,14 +61,14 @@ public: /** Выделяет до указанного количества соединений для работы. * Соединения предоставляют доступ к разным репликам одного шарда. */ - std::vector getMany(Settings * settings = nullptr) override + std::vector getMany(const Settings * settings = nullptr) override { applyLoadBalancing(settings); return Base::getMany(settings); } protected: - bool tryGet(ConnectionPoolPtr pool, Settings * settings, Entry & out_entry, std::stringstream & fail_message) override + bool tryGet(ConnectionPoolPtr pool, const Settings * settings, Entry & out_entry, std::stringstream & fail_message) override { try { @@ -90,7 +90,7 @@ private: std::vector hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов. LoadBalancing default_load_balancing; - void applyLoadBalancing(Settings * settings) + void applyLoadBalancing(const Settings * settings) { LoadBalancing load_balancing = default_load_balancing; if (settings) diff --git a/dbms/include/DB/Client/ParallelReplicas.h b/dbms/include/DB/Client/ParallelReplicas.h index 27e1b7782c1..97881eeec27 100644 --- a/dbms/include/DB/Client/ParallelReplicas.h +++ b/dbms/include/DB/Client/ParallelReplicas.h @@ -21,17 +21,17 @@ class ParallelReplicas final : private boost::noncopyable { public: /// Принимает готовое соединение. - ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_); + ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_); /// Принимает пул, из которого нужно будет достать одно или несколько соединений. - ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_); + ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_); /// Отправить на реплики всё содержимое внешних таблиц. void sendExternalTablesData(std::vector & data); /// Отправить запрос на реплики. void sendQuery(const String & query, const String & query_id = "", - UInt64 stage = QueryProcessingStage::Complete, bool with_pending_data = false); + UInt64 stage = QueryProcessingStage::Complete, bool with_pending_data = false); /// Получить пакет от какой-нибудь реплики. Connection::Packet receivePacket(); @@ -82,7 +82,7 @@ private: void invalidateReplica(ReplicaMap::iterator it); private: - Settings * settings; + const Settings * settings; ReplicaMap replica_map; /// Если не nullptr, то используется, чтобы ограничить сетевой трафик. diff --git a/dbms/include/DB/Columns/ColumnConst.h b/dbms/include/DB/Columns/ColumnConst.h index 3286f707e7e..fad47e76f96 100644 --- a/dbms/include/DB/Columns/ColumnConst.h +++ b/dbms/include/DB/Columns/ColumnConst.h @@ -24,36 +24,51 @@ public: }; -/** шаблон для столбцов-констант (столбцов одинаковых значений). +/** Столбец-константа может содержать внутри себя само значение, + * или, в случае массивов, SharedPtr от значения-массива, + * чтобы избежать проблем производительности при копировании очень больших массивов. + * + * T - тип значения, + * DataHolder - как значение хранится в таблице (либо T, либо SharedPtr) + * Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а. + * + * Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными. */ -template -class ColumnConst final : public IColumnConst +template +class ColumnConstBase : public IColumnConst { +protected: + size_t s; + DataHolder data; + DataTypePtr data_type; + + T & getDataFromHolder() { return static_cast(this)->getDataFromHolderImpl(); } + const T & getDataFromHolder() const { return static_cast(this)->getDataFromHolderImpl(); } + + ColumnConstBase(size_t s_, const DataHolder & data_, DataTypePtr data_type_) + : s(s_), data(data_), data_type(data_type_) {} + public: typedef T Type; typedef typename NearestFieldType::Type FieldType; - /// Для ColumnConst data_type_ должен быть ненулевым. - /// Для ColumnConst data_type_ должен быть ненулевым, если тип данных FixedString. - ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr()) : s(s_), data(data_), data_type(data_type_) {} - std::string getName() const override { return "ColumnConst<" + TypeName::get() + ">"; } bool isNumeric() const override { return IsNumber::value; } bool isFixed() const override { return IsNumber::value; } size_t sizeOfField() const override { return sizeof(T); } - ColumnPtr cloneResized(size_t s_) const override { return new ColumnConst(s_, data, data_type); } + ColumnPtr cloneResized(size_t s_) const override { return new Derived(s_, data, data_type); } size_t size() const override { return s; } - Field operator[](size_t n) const override { return FieldType(data); } - void get(size_t n, Field & res) const override { res = FieldType(data); } + Field operator[](size_t n) const override { return FieldType(getDataFromHolder()); } + void get(size_t n, Field & res) const override { res = FieldType(getDataFromHolder()); } ColumnPtr cut(size_t start, size_t length) const override { - return new ColumnConst(length, data, data_type); + return new Derived(length, data, data_type); } void insert(const Field & x) override { - if (x.get() != FieldType(data)) + if (x.get() != FieldType(getDataFromHolder())) throw Exception("Cannot insert different element into constant column " + getName(), ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN); ++s; @@ -66,7 +81,7 @@ public: void insertFrom(const IColumn & src, size_t n) override { - if (data != static_cast &>(src).data) + if (getDataFromHolder() != static_cast(src).getDataFromHolder()) throw Exception("Cannot insert different element into constant column " + getName(), ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN); ++s; @@ -79,7 +94,7 @@ public: if (s != filt.size()) throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - return new ColumnConst(countBytesInFilter(filt), data, data_type); + return new Derived(countBytesInFilter(filt), data, data_type); } ColumnPtr replicate(const Offsets_t & offsets) const override @@ -88,7 +103,7 @@ public: throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); size_t replicated_size = 0 == s ? 0 : offsets.back(); - return new ColumnConst(replicated_size, data, data_type); + return new Derived(replicated_size, data, data_type); } size_t byteSize() const override { return sizeof(data) + sizeof(s); } @@ -103,13 +118,13 @@ public: if (perm.size() < limit) throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - return new ColumnConst(limit, data, data_type); + return new Derived(limit, data, data_type); } int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { - const ColumnConst & rhs = static_cast &>(rhs_); - return data < rhs.data /// TODO: правильное сравнение NaN-ов в константных столбцах. + const Derived & rhs = static_cast(rhs_); + return getDataFromHolder() < rhs.getDataFromHolder() /// TODO: правильное сравнение NaN-ов в константных столбцах. ? -1 : (data == rhs.data ? 0 @@ -123,30 +138,79 @@ public: res[i] = i; } + DataTypePtr & getDataType() { return data_type; } + const DataTypePtr & getDataType() const { return data_type; } +}; + + +/** шаблон для столбцов-констант (столбцов одинаковых значений). + */ +template +class ColumnConst final : public ColumnConstBase> +{ +private: + friend class ColumnConstBase>; + + T & getDataFromHolderImpl() { return this->data; } + const T & getDataFromHolderImpl() const { return this->data; } + +public: + /// Для ColumnConst data_type_ должен быть ненулевым. + /// Для ColumnConst data_type_ должен быть ненулевым, если тип данных FixedString. + ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr()) + : ColumnConstBase>(s_, data_, data_type_) {} + StringRef getDataAt(size_t n) const override; StringRef getDataAtWithTerminatingZero(size_t n) const override; UInt64 get64(size_t n) const override; /** Более эффективные методы манипуляции */ - T & getData() { return data; } - const T & getData() const { return data; } + T & getData() { return this->data; } + const T & getData() const { return this->data; } /** Преобразование из константы в полноценный столбец */ ColumnPtr convertToFullColumn() const override; void getExtremes(Field & min, Field & max) const override { - min = FieldType(data); - max = FieldType(data); + min = typename ColumnConstBase>::FieldType(this->data); + max = typename ColumnConstBase>::FieldType(this->data); } +}; - DataTypePtr & getDataType() { return data_type; } - const DataTypePtr & getDataType() const { return data_type; } +template <> +class ColumnConst final : public ColumnConstBase, ColumnConst> +{ private: - size_t s; - T data; - DataTypePtr data_type; + friend class ColumnConstBase, ColumnConst>; + + Array & getDataFromHolderImpl() { return *data; } + const Array & getDataFromHolderImpl() const { return *data; } + +public: + /// data_type_ должен быть ненулевым. + ColumnConst(size_t s_, const Array & data_, DataTypePtr data_type_ = DataTypePtr()) + : ColumnConstBase, ColumnConst>(s_, new Array(data_), data_type_) {} + + ColumnConst(size_t s_, const SharedPtr & data_, DataTypePtr data_type_ = DataTypePtr()) + : ColumnConstBase, ColumnConst>(s_, data_, data_type_) {} + + StringRef getDataAt(size_t n) const override; + StringRef getDataAtWithTerminatingZero(size_t n) const override; + UInt64 get64(size_t n) const override; + + /** Более эффективные методы манипуляции */ + const Array & getData() const { return *data; } + + /** Преобразование из константы в полноценный столбец */ + ColumnPtr convertToFullColumn() const override; + + void getExtremes(Field & min, Field & max) const override + { + min = FieldType(); + max = FieldType(); + } }; @@ -158,19 +222,17 @@ template ColumnPtr ColumnConst::convertToFullColumn() const { ColumnVector * res_ = new ColumnVector; ColumnPtr res = res_; - res_->getData().assign(s, data); + res_->getData().assign(this->s, this->data); return res; } template <> ColumnPtr ColumnConst::convertToFullColumn() const; -template <> ColumnPtr ColumnConst::convertToFullColumn() const; - template StringRef ColumnConst::getDataAt(size_t n) const { - throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED); } template <> inline StringRef ColumnConst::getDataAt(size_t n) const @@ -180,7 +242,7 @@ template <> inline StringRef ColumnConst::getDataAt(size_t n) const template UInt64 ColumnConst::get64(size_t n) const { - throw Exception("Method get64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Method get64 is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED); } /// Для элементарных типов. diff --git a/dbms/include/DB/Common/MemoryTracker.h b/dbms/include/DB/Common/MemoryTracker.h index 87f89ac6293..18132397eb7 100644 --- a/dbms/include/DB/Common/MemoryTracker.h +++ b/dbms/include/DB/Common/MemoryTracker.h @@ -38,6 +38,11 @@ public: { return amount; } + + Int64 getPeak() const + { + return peak; + } }; diff --git a/dbms/include/DB/Common/SipHash.h b/dbms/include/DB/Common/SipHash.h index 0536746bd35..3ed015b3f08 100644 --- a/dbms/include/DB/Common/SipHash.h +++ b/dbms/include/DB/Common/SipHash.h @@ -13,6 +13,7 @@ #include #include +#include #define ROTL(x,b) (u64)( ((x) << (b)) | ( (x) >> (64 - (b))) ) diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index ba89beee564..e25fb73fc80 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -78,5 +78,7 @@ /// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций. #define DEFAULT_AIO_FILE_BLOCK_SIZE 4096 +#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR "7500" + #define ALWAYS_INLINE __attribute__((__always_inline__)) #define NO_INLINE __attribute__((__noinline__)) diff --git a/dbms/include/DB/Core/Exception.h b/dbms/include/DB/Core/Exception.h index afc56e6571f..3d28e412137 100644 --- a/dbms/include/DB/Core/Exception.h +++ b/dbms/include/DB/Core/Exception.h @@ -31,6 +31,8 @@ ExceptionPtr cloneCurrentException(); void tryLogCurrentException(const char * log_name); void tryLogCurrentException(Poco::Logger * logger); +std::string getCurrentExceptionMessage(bool with_stacktrace); + void rethrowFirstException(Exceptions & exceptions); diff --git a/dbms/include/DB/DataStreams/BlockIO.h b/dbms/include/DB/DataStreams/BlockIO.h index b0e69bbb27d..06d79cc0016 100644 --- a/dbms/include/DB/DataStreams/BlockIO.h +++ b/dbms/include/DB/DataStreams/BlockIO.h @@ -24,6 +24,25 @@ struct BlockIO Block in_sample; /// Пример блока, который будет прочитан из in. Block out_sample; /// Пример блока, которого нужно писать в out. + /// Здесь могут быть установлены колбэки для логгирования запроса. + std::function finish_callback; + std::function exception_callback; + + /// Вызывайте эти функции, если нужно логгировать запрос. + void onFinish() + { + if (in && finish_callback) + finish_callback(*in); + } + + void onException() + { + if (exception_callback) + exception_callback(); + else + tryLogCurrentException(__PRETTY_FUNCTION__); + } + BlockIO & operator= (const BlockIO & rhs) { /// Обеспечиваем правильный порядок уничтожения. @@ -37,6 +56,9 @@ struct BlockIO in_sample = rhs.in_sample; out_sample = rhs.out_sample; + finish_callback = rhs.finish_callback; + exception_callback = rhs.exception_callback; + return *this; } diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index 90c66edba95..30b044c5488 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -136,6 +136,9 @@ protected: { createParallelReplicas(); + if (settings.skip_unavailable_shards && 0 == parallel_replicas->size()) + return Block(); + established = true; parallel_replicas->sendQuery(query, "", stage, true); diff --git a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h index 440750542f6..39f93e671a6 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h @@ -15,7 +15,7 @@ namespace DB class RemoteBlockOutputStream : public IBlockOutputStream { public: - RemoteBlockOutputStream(Connection & connection_, const String & query_, Settings * settings_ = nullptr) + RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_ = nullptr) : connection(connection_), query(query_), settings(settings_) { } @@ -105,7 +105,7 @@ public: private: Connection & connection; String query; - Settings * settings; + const Settings * settings; Block sample_block; }; diff --git a/dbms/include/DB/DataTypes/FieldToDataType.h b/dbms/include/DB/DataTypes/FieldToDataType.h index 455a9c1218a..846fda28ee2 100644 --- a/dbms/include/DB/DataTypes/FieldToDataType.h +++ b/dbms/include/DB/DataTypes/FieldToDataType.h @@ -4,51 +4,52 @@ #include #include +#include +#include + namespace DB { -/// Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа. +/** Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа. + * В случае, если Field - массив, конвертирует все элементы к общему типу. + */ class FieldToDataType : public StaticVisitor { public: - DataTypePtr operator() (const Null & x) const + DataTypePtr operator() (Null & x) const { - throw Exception("NULL literals is not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED); } - - DataTypePtr operator() (const UInt64 & x) const + + DataTypePtr operator() (UInt64 & x) const { if (x <= std::numeric_limits::max()) return new DataTypeUInt8; if (x <= std::numeric_limits::max()) return new DataTypeUInt16; if (x <= std::numeric_limits::max()) return new DataTypeUInt32; return new DataTypeUInt64; } - - DataTypePtr operator() (const Int64 & x) const + + DataTypePtr operator() (Int64 & x) const { if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return new DataTypeInt8; if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return new DataTypeInt16; if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return new DataTypeInt32; return new DataTypeInt64; } - - DataTypePtr operator() (const Float64 & x) const + + DataTypePtr operator() (Float64 & x) const { return new DataTypeFloat64; } - - DataTypePtr operator() (const String & x) const + + DataTypePtr operator() (String & x) const { return new DataTypeString; } - DataTypePtr operator() (const Array & x) const - { - return new DataTypeArray(apply_visitor(FieldToDataType(), x.at(0))); - } + DataTypePtr operator() (Array & x) const; }; - } diff --git a/dbms/include/DB/Functions/FunctionsHigherOrder.h b/dbms/include/DB/Functions/FunctionsHigherOrder.h index d04bf89620f..8e955410e18 100644 --- a/dbms/include/DB/Functions/FunctionsHigherOrder.h +++ b/dbms/include/DB/Functions/FunctionsHigherOrder.h @@ -41,7 +41,9 @@ struct ArrayMapImpl static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - return new ColumnArray(mapped, array->getOffsetsColumn()); + return mapped->isConst() + ? new ColumnArray(dynamic_cast(*mapped).convertToFullColumn(), array->getOffsetsColumn()) + : new ColumnArray(mapped, array->getOffsetsColumn()); } }; @@ -59,9 +61,20 @@ struct ArrayFilterImpl /// Если массивов несколько, сюда передается первый. static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - ColumnVector * column_filter = typeid_cast *>(&*mapped); + const ColumnVector * column_filter = typeid_cast *>(&*mapped); + if (!column_filter) - throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + { + const ColumnConstUInt8 * column_filter_const = typeid_cast(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getData()) + return array->clone(); + else + return new ColumnArray(array->getDataPtr()->cloneEmpty(), new ColumnArray::ColumnOffsets_t(array->size(), 0)); + } const IColumn::Filter & filter = column_filter->getData(); ColumnPtr filtered = array->getData().filter(filter); @@ -100,9 +113,34 @@ struct ArrayCountImpl static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - ColumnVector * column_filter = typeid_cast *>(&*mapped); + const ColumnVector * column_filter = typeid_cast *>(&*mapped); + if (!column_filter) - throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + { + const ColumnConstUInt8 * column_filter_const = typeid_cast(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getData()) + { + const IColumn::Offsets_t & offsets = array->getOffsets(); + ColumnVector * out_column = new ColumnVector(offsets.size()); + ColumnPtr out_column_ptr = out_column; + ColumnVector::Container_t & out_counts = out_column->getData(); + + size_t pos = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + out_counts[i] = offsets[i] - pos; + pos = offsets[i]; + } + + return out_column_ptr; + } + else + return new ColumnConstUInt32(array->size(), 0); + } const IColumn::Filter & filter = column_filter->getData(); const IColumn::Offsets_t & offsets = array->getOffsets(); @@ -139,9 +177,34 @@ struct ArrayExistsImpl static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - ColumnVector * column_filter = typeid_cast *>(&*mapped); + const ColumnVector * column_filter = typeid_cast *>(&*mapped); + if (!column_filter) - throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + { + const ColumnConstUInt8 * column_filter_const = typeid_cast(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getData()) + { + const IColumn::Offsets_t & offsets = array->getOffsets(); + ColumnVector * out_column = new ColumnVector(offsets.size()); + ColumnPtr out_column_ptr = out_column; + ColumnVector::Container_t & out_exists = out_column->getData(); + + size_t pos = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + out_exists[i] = offsets[i] - pos > 0; + pos = offsets[i]; + } + + return out_column_ptr; + } + else + return new ColumnConstUInt8(array->size(), 0); + } const IColumn::Filter & filter = column_filter->getData(); const IColumn::Offsets_t & offsets = array->getOffsets(); @@ -182,9 +245,34 @@ struct ArrayAllImpl static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - ColumnVector * column_filter = typeid_cast *>(&*mapped); + const ColumnVector * column_filter = typeid_cast *>(&*mapped); + if (!column_filter) - throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + { + const ColumnConstUInt8 * column_filter_const = typeid_cast(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getData()) + return new ColumnConstUInt8(array->size(), 1); + else + { + const IColumn::Offsets_t & offsets = array->getOffsets(); + ColumnVector * out_column = new ColumnVector(offsets.size()); + ColumnPtr out_column_ptr = out_column; + ColumnVector::Container_t & out_all = out_column->getData(); + + size_t pos = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + out_all[i] = offsets[i] == pos; + pos = offsets[i]; + } + + return out_column_ptr; + } + } const IColumn::Filter & filter = column_filter->getData(); const IColumn::Offsets_t & offsets = array->getOffsets(); @@ -245,7 +333,27 @@ struct ArraySumImpl const ColumnVector * column = typeid_cast *>(&*mapped); if (!column) - return false; + { + const ColumnConst * column_const = typeid_cast *>(&*mapped); + + if (!column_const) + return false; + + const Element x = column_const->getData(); + + ColumnVector * res_column = new ColumnVector(offsets.size()); + res_ptr = res_column; + typename ColumnVector::Container_t & res = res_column->getData(); + + size_t pos = 0; + for (size_t i = 0; i < offsets.size(); ++i) + { + res[i] = x * (offsets[i] - pos); + pos = offsets[i]; + } + + return true; + } const typename ColumnVector::Container_t & data = column->getData(); ColumnVector * res_column = new ColumnVector(offsets.size()); @@ -300,9 +408,41 @@ struct ArrayFirstImpl static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - auto column_filter = typeid_cast *>(&*mapped); + auto column_filter = typeid_cast *>(&*mapped); + if (!column_filter) - throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + { + const ColumnConstUInt8 * column_filter_const = typeid_cast(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getData()) + { + const auto & offsets = array->getOffsets(); + const auto & data = array->getData(); + ColumnPtr out{data.cloneEmpty()}; + + size_t pos{}; + for (size_t i = 0; i < offsets.size(); ++i) + { + if (offsets[i] - pos > 0) + out->insert(data[pos]); + else + out->insertDefault(); + + pos = offsets[i]; + } + + return out; + } + else + { + ColumnPtr out{array->getData().cloneEmpty()}; + out->insertDefault(); + return out->replicate(IColumn::Offsets_t(1, array->size())); + } + } const auto & filter = column_filter->getData(); const auto & offsets = array->getOffsets(); @@ -345,9 +485,34 @@ struct ArrayFirstIndexImpl static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped) { - auto column_filter = typeid_cast *>(&*mapped); + auto column_filter = typeid_cast *>(&*mapped); + if (!column_filter) - throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + { + const ColumnConstUInt8 * column_filter_const = typeid_cast(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getData()) + { + const auto & offsets = array->getOffsets(); + auto out_column = new ColumnVector{offsets.size()}; + ColumnPtr out_column_ptr{out_column}; + auto & out_index = out_column->getData(); + + size_t pos{}; + for (size_t i = 0; i < offsets.size(); ++i) + { + out_index[i] = offsets[i] - pos > 0; + pos = offsets[i]; + } + + return out_column_ptr; + } + else + return new ColumnConstUInt32(array->size(), 0); + } const auto & filter = column_filter->getData(); const auto & offsets = array->getOffsets(); diff --git a/dbms/include/DB/Functions/FunctionsRound.h b/dbms/include/DB/Functions/FunctionsRound.h index 7706c86519b..bf321e8d760 100644 --- a/dbms/include/DB/Functions/FunctionsRound.h +++ b/dbms/include/DB/Functions/FunctionsRound.h @@ -89,11 +89,12 @@ namespace DB static inline ResultType apply(A x) { - return x < 18 ? 0 + return x < 1 ? 0 + : (x < 18 ? 17 : (x < 25 ? 18 : (x < 35 ? 25 : (x < 45 ? 35 - : 45))); + : 45)))); } }; diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 351fb47f1ef..f1a5a3a45a1 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -447,10 +447,15 @@ struct MatchImpl const char * str_data = reinterpret_cast(&data[i != 0 ? offsets[i - 1] : 0]); size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1; + /** Даже в случае required_substring_is_prefix используем UNANCHORED проверку регекспа, + * чтобы он мог сматчиться, когда required_substring встречается в строке несколько раз, + * и на первом вхождении регексп не матчит. + */ + if (required_substring_is_prefix) res[i] = revert ^ regexp->getRE2()->Match( re2_st::StringPiece(str_data, str_size), - reinterpret_cast(pos) - str_data, str_size, re2_st::RE2::ANCHOR_START, nullptr, 0); + reinterpret_cast(pos) - str_data, str_size, re2_st::RE2::UNANCHORED, nullptr, 0); else res[i] = revert ^ regexp->getRE2()->Match( re2_st::StringPiece(str_data, str_size), diff --git a/dbms/include/DB/IO/ReadBuffer.h b/dbms/include/DB/IO/ReadBuffer.h index 6366bbe6b01..97442d69717 100644 --- a/dbms/include/DB/IO/ReadBuffer.h +++ b/dbms/include/DB/IO/ReadBuffer.h @@ -84,7 +84,7 @@ public: void ignore(size_t n) { - while (!eof() && n != 0) + while (n != 0 && !eof()) { size_t bytes_to_ignore = std::min(static_cast(working_buffer.end() - pos), n); pos += bytes_to_ignore; diff --git a/dbms/include/DB/IO/ReadHelpers.h b/dbms/include/DB/IO/ReadHelpers.h index ef4a9eeed0b..12302a33436 100644 --- a/dbms/include/DB/IO/ReadHelpers.h +++ b/dbms/include/DB/IO/ReadHelpers.h @@ -105,6 +105,7 @@ void readVectorBinary(std::vector & v, ReadBuffer & buf, size_t MAX_VECTOR_SI void assertString(const char * s, ReadBuffer & buf); void assertEOF(ReadBuffer & buf); +void assertChar(char symbol, ReadBuffer & buf); inline void assertString(const String & s, ReadBuffer & buf) { diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index 8e2b73dcd86..08e6e43da9a 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.h @@ -252,13 +252,19 @@ void writeAnyEscapedString(const String & s, WriteBuffer & buf) } -inline void writeEscapedString(const String & s, WriteBuffer & buf) +inline void writeEscapedString(const char * str, size_t size, WriteBuffer & buf) { /// strpbrk в libc под Linux на процессорах с SSE 4.2 хорошо оптимизирована (этот if ускоряет код в 1.5 раза) - if (nullptr == strpbrk(s.data(), "\b\f\n\r\t\'\\") && strlen(s.data()) == s.size()) - writeString(s, buf); + if (nullptr == strpbrk(str, "\b\f\n\r\t\'\\") && strlen(str) == size) + writeString(str, size, buf); else - writeAnyEscapedString<'\''>(s, buf); + writeAnyEscapedString<'\''>(str, str + size, buf); +} + + +inline void writeEscapedString(const String & s, WriteBuffer & buf) +{ + writeEscapedString(s.data(), s.size(), buf); } @@ -470,6 +476,10 @@ inline void writeText(const Float32 & x, WriteBuffer & buf) { writeFloatText(x, inline void writeText(const Float64 & x, WriteBuffer & buf) { writeFloatText(x, buf); } inline void writeText(const String & x, WriteBuffer & buf) { writeEscapedString(x, buf); } inline void writeText(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); } +/// в отличие от метода для std::string +/// здесь предполагается, что x null-terminated строка. +inline void writeText(const char * x, WriteBuffer & buf) { writeEscapedString(x, strlen(x), buf); } +inline void writeText(const char * x, size_t size, WriteBuffer & buf) { writeEscapedString(x, size, buf); } inline void writeText(const VisitID_t & x, WriteBuffer & buf) { writeIntText(static_cast(x), buf); } inline void writeText(const mysqlxx::Date & x, WriteBuffer & buf) { writeDateText(x, buf); } diff --git a/dbms/include/DB/Interpreters/Context.h b/dbms/include/DB/Interpreters/Context.h index f646c627ecb..2f62242ae8a 100644 --- a/dbms/include/DB/Interpreters/Context.h +++ b/dbms/include/DB/Interpreters/Context.h @@ -40,6 +40,7 @@ class ProcessListElement; class Macros; class Progress; class Clusters; +class QueryLog; /// имя таблицы -> таблица @@ -64,12 +65,30 @@ typedef std::vector Dependencies; */ class Context { +public: + enum class Interface + { + TCP = 1, + HTTP = 2, + OLAP_HTTP = 3, + }; + + enum class HTTPMethod + { + UNKNOWN = 0, + GET = 1, + POST = 2, + }; + private: typedef std::shared_ptr Shared; Shared shared; String user; /// Текущий пользователь. Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос. + Interface interface = Interface::TCP; + HTTPMethod http_method = HTTPMethod::UNKNOWN; /// NOTE Возможно, перенести это в отдельный struct ClientInfo. + std::shared_ptr quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает. String current_database; /// Текущая БД. String current_query_id; /// Id текущего запроса. @@ -108,6 +127,12 @@ public: String getUser() const { return user; } Poco::Net::IPAddress getIPAddress() const { return ip_address; } + Interface getInterface() const { return interface; } + void setInterface(Interface interface_) { interface = interface_; } + + HTTPMethod getHTTPMethod() const { return http_method; } + void setHTTPMethod(HTTPMethod http_method_) { http_method = http_method_; } + void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); QuotaForIntervals & getQuota(); @@ -242,6 +267,8 @@ public: Compiler & getCompiler(); + QueryLog & getQueryLog(); + /// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле. CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const; diff --git a/dbms/include/DB/Interpreters/QueryLog.h b/dbms/include/DB/Interpreters/QueryLog.h index 50f21c31b34..b6ade5d8fca 100644 --- a/dbms/include/DB/Interpreters/QueryLog.h +++ b/dbms/include/DB/Interpreters/QueryLog.h @@ -5,8 +5,8 @@ #include #include #include +#include #include -#include namespace DB @@ -34,40 +34,33 @@ struct QueryLogElement SHUTDOWN = 0, /// Эта запись имеет служебное значение. QUERY_START = 1, QUERY_FINISH = 2, + EXCEPTION_BEFORE_START = 3, + EXCEPTION_WHILE_PROCESSING = 4, }; - enum Interface - { - TCP = 1, - HTTP = 2, - OLAP_HTTP = 3, - }; - - enum HTTPMethod - { - UNKNOWN = 0, - GET = 1, - POST = 2, - }; - - Type type; + Type type = QUERY_START; /// В зависимости от типа, не все поля могут быть заполнены. - time_t event_time; - time_t query_start_time; - UInt64 query_duration_ms; + time_t event_time{}; + time_t query_start_time{}; + UInt64 query_duration_ms{}; - UInt64 read_rows; - UInt64 read_bytes; + UInt64 read_rows{}; + UInt64 read_bytes{}; - UInt64 result_rows; - UInt64 result_bytes; + UInt64 result_rows{}; + UInt64 result_bytes{}; + + UInt64 memory_usage{}; String query; - Interface interface; - HTTPMethod http_method; + String exception; + String stack_trace; + + Context::Interface interface = Context::Interface::TCP; + Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN; Poco::Net::IPAddress ip_address; String user; String query_id; @@ -76,6 +69,8 @@ struct QueryLogElement #define DBMS_QUERY_LOG_QUEUE_SIZE 1024 +class Context; + class QueryLog : private boost::noncopyable { @@ -89,25 +84,8 @@ public: * где N - минимальное число, начиная с 1 такое, что таблицы с таким именем ещё нет; * и создаётся новая таблица, как будто существующей таблицы не было. */ - QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_) - : context(context_), database_name(database_name_), table_name(table_name_), flush_interval_milliseconds(flush_interval_milliseconds_) - { - data.reserve(DBMS_QUERY_LOG_QUEUE_SIZE); - - // TODO - - saving_thread = std::thread([this] { threadFunction(); }); - } - - ~QueryLog() - { - /// Говорим потоку, что надо завершиться. - QueryLogElement elem; - elem.type = QueryLogElement::SHUTDOWN; - queue.push(elem); - - saving_thread.join(); - } + QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_); + ~QueryLog(); /** Добавить запись в лог. * Сохранение в таблицу делается асинхронно, и в случае сбоя, запись может никуда не попасть. @@ -134,111 +112,15 @@ private: */ std::vector data; + Logger * log {&Logger::get("QueryLog")}; + /** В этом потоке данные вынимаются из queue, складываются в data, а затем вставляются в таблицу. */ std::thread saving_thread; - - void threadFunction() - { - Stopwatch time_after_last_write; - bool first = true; - - while (true) - { - try - { - if (first) - { - time_after_last_write.restart(); - first = false; - } - - QueryLogElement element; - bool has_element = false; - - if (data.empty()) - { - element = queue.pop(); - has_element = true; - } - else - { - size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000; - if (milliseconds_elapsed < flush_interval_milliseconds) - has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed); - } - - if (has_element) - { - if (element.type = QueryLogElement::SHUTDOWN) - { - flush(); - break; - } - else - data.push_back(element); - } - - size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000; - if (milliseconds_elapsed >= flush_interval_milliseconds) - { - /// Записываем данные в таблицу. - flush(); - time_after_last_write.restart(); - } - } - catch (...) - { - /// В случае ошибки теряем накопленные записи, чтобы не блокироваться. - data.clear(); - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - } - - Block createBlock() - { - return { - {new ColumnUInt8, new DataTypeUInt8, "type"}, - {new ColumnUInt32, new DataTypeDateTime, "event_time"}, - {new ColumnUInt32, new DataTypeDateTime, "query_start_time"}, - }; - - /* time_t event_time; - time_t query_start_time; - UInt64 query_duration_ms; - - UInt64 read_rows; - UInt64 read_bytes; - - UInt64 result_rows; - UInt64 result_bytes; - - String query; - - Interface interface; - HTTPMethod http_method; - Poco::Net::IPAddress ip_address; - String user; - String query_id;*/ - } - - void flush() - { - try - { - Block block = createBlock(); - - // TODO Формирование блока и запись. - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - data.clear(); - } + void threadFunction(); + static Block createBlock(); + void flush(); }; diff --git a/dbms/include/DB/Interpreters/Quota.h b/dbms/include/DB/Interpreters/Quota.h index 4c03ecd41a6..ea8e86d8450 100644 --- a/dbms/include/DB/Interpreters/Quota.h +++ b/dbms/include/DB/Interpreters/Quota.h @@ -83,7 +83,7 @@ struct QuotaForInterval /// Увеличить соответствующее значение. void addQuery(time_t current_time, const String & quota_name); - void addError(time_t current_time, const String & quota_name); + void addError(time_t current_time, const String & quota_name) noexcept; /// Проверить, не превышена ли квота уже. Если превышена - кидает исключение. void checkExceeded(time_t current_time, const String & quota_name); @@ -139,7 +139,7 @@ public: void setMax(const QuotaForIntervals & quota); void addQuery(time_t current_time); - void addError(time_t current_time); + void addError(time_t current_time) noexcept; void checkExceeded(time_t current_time); diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 8d799249f71..2bc4c0bca4a 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -97,6 +97,14 @@ struct Settings M(SettingUInt64, parallel_replicas_count, 0) \ M(SettingUInt64, parallel_replica_offset, 0) \ \ + /** Тихо пропускать недоступные шарды. */ \ + M(SettingBool, skip_unavailable_shards, false) \ + \ + /** Не мерджить состояния агрегации с разных серверов при распределённой обработке запроса \ + * - на случай, когда доподлинно известно, что на разных шардах разные ключи. \ + */ \ + M(SettingBool, distributed_group_by_no_merge, false) \ + \ /** Тонкие настройки для чтения из MergeTree */ \ \ /** Если из одного файла читается хотя бы столько строк, чтение можно распараллелить. */ \ @@ -135,6 +143,9 @@ struct Settings \ /** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \ M(SettingUInt64, priority, 0) \ + \ + /** Логгировать запросы и писать лог в системную таблицу. */ \ + M(SettingBool, log_queries, 0) \ /// Всевозможные ограничения на выполнение запроса. Limits limits; diff --git a/dbms/include/DB/Parsers/ASTLiteral.h b/dbms/include/DB/Parsers/ASTLiteral.h index b7b24ef56db..2a610255be8 100644 --- a/dbms/include/DB/Parsers/ASTLiteral.h +++ b/dbms/include/DB/Parsers/ASTLiteral.h @@ -20,7 +20,7 @@ public: ASTLiteral() = default; ASTLiteral(const StringRange range_, const Field & value_) : ASTWithAlias(range_), value(value_) {} - String getColumnName() const override { return apply_visitor(FieldVisitorToString(), value); } + String getColumnName() const override; /** Получить текст, который идентифицирует этот элемент. */ String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); } diff --git a/dbms/include/DB/Parsers/ASTQueryWithOutput.h b/dbms/include/DB/Parsers/ASTQueryWithOutput.h index 2b5723249f2..92cba621eab 100644 --- a/dbms/include/DB/Parsers/ASTQueryWithOutput.h +++ b/dbms/include/DB/Parsers/ASTQueryWithOutput.h @@ -5,10 +5,9 @@ namespace DB { - - + /** Запрос с секцией FORMAT. - */ + */ class ASTQueryWithOutput : public IAST { public: @@ -16,6 +15,11 @@ public: ASTQueryWithOutput() = default; ASTQueryWithOutput(const StringRange range_) : IAST(range_) {} + + /** Возвращает указатель на формат. Если типом объекта является ASTSelectQuery, + * то эта функция возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL. + */ + virtual const IAST * getFormat() const { return format.get(); } }; diff --git a/dbms/include/DB/Parsers/ASTSelectQuery.h b/dbms/include/DB/Parsers/ASTSelectQuery.h index 2d5f9cba3a2..095e9436238 100644 --- a/dbms/include/DB/Parsers/ASTSelectQuery.h +++ b/dbms/include/DB/Parsers/ASTSelectQuery.h @@ -34,6 +34,9 @@ public: ASTPtr clone() const override; + /// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL. + const IAST * getFormat() const override; + public: bool distinct = false; ASTPtr select_expression_list; diff --git a/dbms/include/DB/Parsers/ExpressionElementParsers.h b/dbms/include/DB/Parsers/ExpressionElementParsers.h index 6ea723f6742..e060a47fc17 100644 --- a/dbms/include/DB/Parsers/ExpressionElementParsers.h +++ b/dbms/include/DB/Parsers/ExpressionElementParsers.h @@ -100,6 +100,20 @@ protected: }; +/** Массив литералов. + * Массивы могут распарситься и как применение оператора []. + * Но парсинг всего массива как целой константы серьёзно ускоряет анализ выражений в случае очень больших массивов. + * Мы пробуем распарсить массив как массив литералов сначала (fast path), + * а если не получилось (когда массив состоит из сложных выражений) - парсим как применение оператора [] (slow path). + */ +class ParserArrayOfLiterals : public IParserBase +{ +protected: + const char * getName() const { return "array"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected); +}; + + /** Литерал - одно из: NULL, UInt64, Int64, Float64, String. */ class ParserLiteral : public IParserBase diff --git a/dbms/include/DB/Storages/MergeTree/PKCondition.h b/dbms/include/DB/Storages/MergeTree/PKCondition.h index 4e33612fc9f..47d34413db3 100644 --- a/dbms/include/DB/Storages/MergeTree/PKCondition.h +++ b/dbms/include/DB/Storages/MergeTree/PKCondition.h @@ -15,115 +15,13 @@ namespace DB { -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wsign-compare" - -/** Более точное сравнение. - * Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой. - * Правила сравнения - такие же, что и в FunctionsComparison. - * В том числе, сравнение знаковых и беззнаковых оставляем UB. - */ -class FieldVisitorAccurateEquals : public StaticVisitor -{ -public: - bool operator() (const Null & l, const Null & r) const { return true; } - bool operator() (const Null & l, const UInt64 & r) const { return false; } - bool operator() (const Null & l, const Int64 & r) const { return false; } - bool operator() (const Null & l, const Float64 & r) const { return false; } - bool operator() (const Null & l, const String & r) const { return false; } - bool operator() (const Null & l, const Array & r) const { return false; } - - bool operator() (const UInt64 & l, const Null & r) const { return false; } - bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const String & r) const { return false; } - bool operator() (const UInt64 & l, const Array & r) const { return false; } - - bool operator() (const Int64 & l, const Null & r) const { return false; } - bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const Int64 & l, const Int64 & r) const { return l == r; } - bool operator() (const Int64 & l, const Float64 & r) const { return l == r; } - bool operator() (const Int64 & l, const String & r) const { return false; } - bool operator() (const Int64 & l, const Array & r) const { return false; } - - bool operator() (const Float64 & l, const Null & r) const { return false; } - bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const Float64 & l, const Int64 & r) const { return l == r; } - bool operator() (const Float64 & l, const Float64 & r) const { return l == r; } - bool operator() (const Float64 & l, const String & r) const { return false; } - bool operator() (const Float64 & l, const Array & r) const { return false; } - - bool operator() (const String & l, const Null & r) const { return false; } - bool operator() (const String & l, const UInt64 & r) const { return false; } - bool operator() (const String & l, const Int64 & r) const { return false; } - bool operator() (const String & l, const Float64 & r) const { return false; } - bool operator() (const String & l, const String & r) const { return l == r; } - bool operator() (const String & l, const Array & r) const { return false; } - - bool operator() (const Array & l, const Null & r) const { return false; } - bool operator() (const Array & l, const UInt64 & r) const { return false; } - bool operator() (const Array & l, const Int64 & r) const { return false; } - bool operator() (const Array & l, const Float64 & r) const { return false; } - bool operator() (const Array & l, const String & r) const { return false; } - bool operator() (const Array & l, const Array & r) const { return l == r; } -}; - -class FieldVisitorAccurateLess : public StaticVisitor -{ -public: - bool operator() (const Null & l, const Null & r) const { return false; } - bool operator() (const Null & l, const UInt64 & r) const { return true; } - bool operator() (const Null & l, const Int64 & r) const { return true; } - bool operator() (const Null & l, const Float64 & r) const { return true; } - bool operator() (const Null & l, const String & r) const { return true; } - bool operator() (const Null & l, const Array & r) const { return true; } - - bool operator() (const UInt64 & l, const Null & r) const { return false; } - bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const String & r) const { return true; } - bool operator() (const UInt64 & l, const Array & r) const { return true; } - - bool operator() (const Int64 & l, const Null & r) const { return false; } - bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const Int64 & l, const Int64 & r) const { return l < r; } - bool operator() (const Int64 & l, const Float64 & r) const { return l < r; } - bool operator() (const Int64 & l, const String & r) const { return true; } - bool operator() (const Int64 & l, const Array & r) const { return true; } - - bool operator() (const Float64 & l, const Null & r) const { return false; } - bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const Float64 & l, const Int64 & r) const { return l < r; } - bool operator() (const Float64 & l, const Float64 & r) const { return l < r; } - bool operator() (const Float64 & l, const String & r) const { return true; } - bool operator() (const Float64 & l, const Array & r) const { return true; } - - bool operator() (const String & l, const Null & r) const { return false; } - bool operator() (const String & l, const UInt64 & r) const { return false; } - bool operator() (const String & l, const Int64 & r) const { return false; } - bool operator() (const String & l, const Float64 & r) const { return false; } - bool operator() (const String & l, const String & r) const { return l < r; } - bool operator() (const String & l, const Array & r) const { return true; } - - bool operator() (const Array & l, const Null & r) const { return false; } - bool operator() (const Array & l, const UInt64 & r) const { return false; } - bool operator() (const Array & l, const Int64 & r) const { return false; } - bool operator() (const Array & l, const Float64 & r) const { return false; } - bool operator() (const Array & l, const String & r) const { return false; } - bool operator() (const Array & l, const Array & r) const { return l < r; } -}; - -#pragma GCC diagnostic pop - /** Диапазон с открытыми или закрытыми концами; возможно, неограниченный. */ struct Range { private: - static bool equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); } - static bool less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); } + static bool equals(const Field & lhs, const Field & rhs); + static bool less(const Field & lhs, const Field & rhs); public: Field left; /// левая граница, если есть diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 103ba626a4c..88e087a7457 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -96,6 +96,7 @@ private: }; bool is_interactive = true; /// Использовать readline интерфейс или batch режим. + bool need_render_progress = true; /// Рисовать прогресс выполнения запроса. bool print_time_to_stderr = false; /// В неинтерактивном режиме, выводить время выполнения в stderr. bool stdin_is_not_tty = false; /// stdin - не терминал. @@ -268,6 +269,9 @@ private: insert_format = "Values"; insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); + if (!is_interactive) + need_render_progress = config().getBool("progress", false); + connect(); if (is_interactive) @@ -850,11 +854,11 @@ private: /// Формат может быть указан в запросе. if (ASTQueryWithOutput * query_with_output = dynamic_cast(&*parsed_query)) { - if (query_with_output->format) + if (query_with_output->getFormat() != nullptr) { if (has_vertical_output_suffix) throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED); - if (ASTIdentifier * id = typeid_cast(&*query_with_output->format)) + if (const ASTIdentifier * id = typeid_cast(query_with_output->getFormat())) current_format = id->name; } } @@ -906,7 +910,7 @@ private: void writeProgress() { - if (!is_interactive) + if (!need_render_progress) return; static size_t increment = 0; @@ -1053,6 +1057,7 @@ public: ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") ("stacktrace", "print stack traces of exceptions") + ("progress", "print progress even in non-interactive mode") APPLY_FOR_SETTINGS(DECLARE_SETTING) APPLY_FOR_LIMITS(DECLARE_LIMIT) ; @@ -1171,6 +1176,8 @@ public: config().setBool("vertical", true); if (options.count("stacktrace")) config().setBool("stacktrace", true); + if (options.count("progress")) + config().setBool("progress", true); if (options.count("time")) print_time_to_stderr = true; } diff --git a/dbms/src/Client/ParallelReplicas.cpp b/dbms/src/Client/ParallelReplicas.cpp index 81d629d392c..135c1b06aa0 100644 --- a/dbms/src/Client/ParallelReplicas.cpp +++ b/dbms/src/Client/ParallelReplicas.cpp @@ -3,7 +3,7 @@ namespace DB { -ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_) +ParallelReplicas::ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_) : settings(settings_), throttler(throttler_), active_replica_count(1), supports_parallel_execution(false) @@ -11,7 +11,7 @@ ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings registerReplica(connection_); } -ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_) +ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_) : settings(settings_), throttler(throttler_) { if (pool_ == nullptr) @@ -37,7 +37,8 @@ ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_ supports_parallel_execution = false; pool_entry = pool_->get(settings); - registerReplica(&*pool_entry); + if (!pool_entry.isNull()) + registerReplica(&*pool_entry); } } diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 0a8c1170ce2..a121b02fb97 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -1,5 +1,6 @@ -#include +#include #include +#include #include #include @@ -61,7 +62,7 @@ template <> ColumnPtr ColumnConst::convertToFullColumn() const } -template <> ColumnPtr ColumnConst::convertToFullColumn() const +ColumnPtr ColumnConst::convertToFullColumn() const { if (!data_type) throw Exception("No data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR); @@ -70,7 +71,8 @@ template <> ColumnPtr ColumnConst::convertToFullColumn() const if (!type) throw Exception("Non-array data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR); - size_t array_size = data.size(); + const Array & array = getDataFromHolderImpl(); + size_t array_size = array.size(); ColumnPtr nested_column = type->getNestedType()->createColumn(); ColumnArray * res = new ColumnArray(nested_column); @@ -81,11 +83,27 @@ template <> ColumnPtr ColumnConst::convertToFullColumn() const { offsets[i] = (i + 1) * array_size; for (size_t j = 0; j < array_size; ++j) - nested_column->insert(data[j]); + nested_column->insert(array[j]); } return res; } +StringRef ColumnConst::getDataAt(size_t n) const +{ + throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED); +} + +UInt64 ColumnConst::get64(size_t n) const +{ + throw Exception("Method get64 is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED); +} + +StringRef ColumnConst::getDataAtWithTerminatingZero(size_t n) const +{ + throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED); +} + + } diff --git a/dbms/src/Core/Exception.cpp b/dbms/src/Core/Exception.cpp index f5864a4f210..789af7beeae 100644 --- a/dbms/src/Core/Exception.cpp +++ b/dbms/src/Core/Exception.cpp @@ -59,6 +59,19 @@ void tryLogCurrentException(const char * log_name) void tryLogCurrentException(Poco::Logger * logger) { + try + { + LOG_ERROR(logger, getCurrentExceptionMessage(true)); + } + catch (...) + { + } +} + +std::string getCurrentExceptionMessage(bool with_stacktrace) +{ + std::stringstream stream; + try { throw; @@ -67,8 +80,10 @@ void tryLogCurrentException(Poco::Logger * logger) { try { - LOG_ERROR(logger, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() - << ", Stack trace:\n\n" << e.getStackTrace().toString()); + stream << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); + + if (with_stacktrace) + stream << ", Stack trace:\n\n" << e.getStackTrace().toString(); } catch (...) {} } @@ -76,8 +91,8 @@ void tryLogCurrentException(Poco::Logger * logger) { try { - LOG_ERROR(logger, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() - << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()); + stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() + << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); } catch (...) {} } @@ -91,7 +106,7 @@ void tryLogCurrentException(Poco::Logger * logger) if (status) name += " (demangling status: " + toString(status) + ")"; - LOG_ERROR(logger, "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what()); + stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what(); } catch (...) {} } @@ -105,10 +120,12 @@ void tryLogCurrentException(Poco::Logger * logger) if (status) name += " (demangling status: " + toString(status) + ")"; - LOG_ERROR(logger, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name); + stream << "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name; } catch (...) {} } + + return stream.str(); } diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp new file mode 100644 index 00000000000..52e0d611b8f --- /dev/null +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -0,0 +1,170 @@ +#include + +namespace DB +{ + + +template +static void convertArrayToCommonType(Array & arr) +{ + for (auto & elem : arr) + elem = apply_visitor(FieldVisitorConvertToNumber(), elem); +} + + +DataTypePtr FieldToDataType::operator() (Array & x) const +{ + if (x.empty()) + throw Exception("Cannot infer type of empty array", ErrorCodes::EMPTY_DATA_PASSED); + + /** Тип массива нужно вывести по типу его элементов. + * Если элементы - числа, то нужно выбрать наименьший общий тип, если такой есть, + * или кинуть исключение. + * Код похож на NumberTraits::ResultOfIf, но тем кодом трудно здесь непосредственно воспользоваться. + * + * Также заметим, что Float32 не выводится, вместо этого используется только Float64. + * Это сделано потому что литералов типа Float32 не бывает в запросе. + */ + + bool has_string = false; + bool has_array = false; + bool has_float = false; + int max_bits = 0; + int max_signed_bits = 0; + int max_unsigned_bits = 0; + + for (const Field & elem : x) + { + switch (elem.getType()) + { + case Field::Types::UInt64: + { + UInt64 num = elem.get(); + if (num <= std::numeric_limits::max()) + max_unsigned_bits = std::max(8, max_unsigned_bits); + else if (num <= std::numeric_limits::max()) + max_unsigned_bits = std::max(16, max_unsigned_bits); + else if (num <= std::numeric_limits::max()) + max_unsigned_bits = std::max(32, max_unsigned_bits); + else + max_unsigned_bits = 64; + max_bits = std::max(max_unsigned_bits, max_bits); + break; + } + case Field::Types::Int64: + { + Int64 num = elem.get(); + if (num <= std::numeric_limits::max() && num >= std::numeric_limits::min()) + max_signed_bits = std::max(8, max_signed_bits); + else if (num <= std::numeric_limits::max() && num >= std::numeric_limits::min()) + max_signed_bits = std::max(16, max_signed_bits); + else if (num <= std::numeric_limits::max() && num >= std::numeric_limits::min()) + max_signed_bits = std::max(32, max_signed_bits); + else + max_signed_bits = 64; + max_bits = std::max(max_signed_bits, max_bits); + break; + } + case Field::Types::Float64: + { + has_float = true; + break; + } + case Field::Types::String: + { + has_string = true; + break; + } + case Field::Types::Array: + { + has_array = true; + break; + } + case Field::Types::Null: + { + throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + break; + } + } + } + + if ((has_string + has_array + (max_bits > 0)) > 1) + throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (has_array) + throw Exception("Type inference of multidimensional arrays is not supported", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (has_string) + return new DataTypeArray(new DataTypeString); + + if (has_float && max_bits == 64) + throw Exception("Incompatible types Float64 and UInt64/Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (has_float) + { + convertArrayToCommonType(x); + return new DataTypeArray(new DataTypeFloat64); + } + + if (max_signed_bits == 64 && max_unsigned_bits == 64) + throw Exception("Incompatible types UInt64 and Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (max_signed_bits && !max_unsigned_bits) + { + if (max_signed_bits == 8) + return new DataTypeArray(new DataTypeInt8); + if (max_signed_bits == 16) + return new DataTypeArray(new DataTypeInt16); + if (max_signed_bits == 32) + return new DataTypeArray(new DataTypeInt32); + if (max_signed_bits == 64) + return new DataTypeArray(new DataTypeInt64); + } + + if (!max_signed_bits && max_unsigned_bits) + { + if (max_unsigned_bits == 8) + return new DataTypeArray(new DataTypeUInt8); + if (max_unsigned_bits == 16) + return new DataTypeArray(new DataTypeUInt16); + if (max_unsigned_bits == 32) + return new DataTypeArray(new DataTypeUInt32); + if (max_unsigned_bits == 64) + return new DataTypeArray(new DataTypeUInt64); + } + + if (max_signed_bits && max_unsigned_bits) + { + convertArrayToCommonType(x); + + if (max_unsigned_bits >= max_signed_bits) + { + /// Беззнаковый тип не помещается в знаковый. Надо увеличить количество бит. + if (max_bits == 8) + return new DataTypeArray(new DataTypeInt16); + if (max_bits == 16) + return new DataTypeArray(new DataTypeInt32); + if (max_bits == 32) + return new DataTypeArray(new DataTypeInt64); + else + throw Exception("Incompatible types UInt64 and signed integer of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + else + { + /// Беззнаковый тип помещается в знаковый. + if (max_bits == 8) + return new DataTypeArray(new DataTypeInt8); + if (max_bits == 16) + return new DataTypeArray(new DataTypeInt16); + if (max_bits == 32) + return new DataTypeArray(new DataTypeInt32); + if (max_bits == 64) + return new DataTypeArray(new DataTypeInt64); + } + } + + throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); +} + + +} diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index c392ba094c8..c16d547ca9f 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -36,6 +36,16 @@ void assertString(const char * s, ReadBuffer & buf) } } +void assertChar(char symbol, ReadBuffer & buf) +{ + if (buf.eof() || *buf.position() != symbol) + { + char err[2] = {symbol, '\0'}; + throwAtAssertionFailed(err, buf); + } + ++buf.position(); +} + void assertEOF(ReadBuffer & buf) { if (!buf.eof()) diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 9d0dc69c36e..100b846d5fb 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -557,14 +557,21 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result, for (size_t i = 0; i < aggregates_size; ++i) aggregate_columns[i].resize(aggregates[i].arguments.size()); + /** Константные столбцы не поддерживаются напрямую при агрегации. + * Чтобы они всё-равно работали, материализуем их. + */ + Columns materialized_columns; + /// Запоминаем столбцы, с которыми будем работать for (size_t i = 0; i < keys_size; ++i) { key_columns[i] = block.getByPosition(keys[i]).column; - if (key_columns[i]->isConst()) - throw Exception("Constants are not allowed as GROUP BY keys" - " (but all of them must be eliminated in ExpressionAnalyzer)", ErrorCodes::ILLEGAL_COLUMN); + if (const IColumnConst * column_const = dynamic_cast(key_columns[i])) + { + materialized_columns.push_back(column_const->convertToFullColumn()); + key_columns[i] = materialized_columns.back().get(); + } } for (size_t i = 0; i < aggregates_size; ++i) @@ -573,11 +580,11 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result, { aggregate_columns[i][j] = block.getByPosition(aggregates[i].arguments[j]).column; - /** Агрегатные функции рассчитывают, что в них передаются полноценные столбцы. - * Поэтому, стобцы-константы не разрешены в качестве аргументов агрегатных функций. - */ - if (aggregate_columns[i][j]->isConst()) - throw Exception("Constants are not allowed as arguments of aggregate functions", ErrorCodes::ILLEGAL_COLUMN); + if (const IColumnConst * column_const = dynamic_cast(aggregate_columns[i][j])) + { + materialized_columns.push_back(column_const->convertToFullColumn()); + aggregate_columns[i][j] = materialized_columns.back().get(); + } } } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index c67c36e2cd0..fbde5ad7de6 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +87,7 @@ struct ContextShared BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами. Macros macros; /// Подстановки из конфига. std::unique_ptr compiler; /// Для динамической компиляции частей запроса, при необходимости. + std::unique_ptr query_log; /// Для логгирования запросов. mutable std::unique_ptr compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска. /// Кластеры для distributed таблиц @@ -830,6 +832,26 @@ Compiler & Context::getCompiler() } +QueryLog & Context::getQueryLog() +{ + Poco::ScopedLock lock(shared->mutex); + + if (!shared->query_log) + { + auto & config = Poco::Util::Application::instance().config(); + + String database = config.getString("query_log.database", "system"); + String table = config.getString("query_log.table", "query_log"); + size_t flush_interval_milliseconds = parse( + config.getString("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR)); + + shared->query_log.reset(new QueryLog{ *this, database, table, flush_interval_milliseconds }); + } + + return *shared->query_log; +} + + CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part_size_ratio) const { Poco::ScopedLock lock(shared->mutex); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 56cfd614b8c..60d7b502fb0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1399,6 +1399,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl else if (ASTLiteral * node = typeid_cast(&*ast)) { DataTypePtr type = apply_visitor(FieldToDataType(), node->value); + ColumnWithNameAndType column; column.column = type->createConstColumn(1, node->value); column.type = type; diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 2f2d8e2cb32..247c5e99a76 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -81,6 +81,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) { const std::lock_guard lock{dictionaries_mutex}; + const auto & lifetime = dict_ptr->getLifetime(); + std::uniform_int_distribution distribution{lifetime.min_sec, lifetime.max_sec}; + update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; + const auto dict_it = dictionaries.find(name); if (dict_it->second.dict) dict_it->second.dict->set(dict_ptr.release()); @@ -90,10 +94,6 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error) /// erase stored exception on success dict_it->second.exception = std::exception_ptr{}; - const auto & lifetime = dict_ptr->getLifetime(); - std::uniform_int_distribution distribution{lifetime.min_sec, lifetime.max_sec}; - update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}; - recreated_failed_dictionaries.push_back(name); } } diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp new file mode 100644 index 00000000000..7e342b64343 --- /dev/null +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -0,0 +1,279 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +QueryLog::QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_) + : context(context_), database_name(database_name_), table_name(table_name_), flush_interval_milliseconds(flush_interval_milliseconds_) +{ + data.reserve(DBMS_QUERY_LOG_QUEUE_SIZE); + + { + String description = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name); + + Poco::ScopedLock lock(context.getMutex()); + + table = context.tryGetTable(database_name, table_name); + + if (table) + { + const Block expected = createBlock(); + const Block actual = table->getSampleBlockNonMaterialized(); + + if (!blocksHaveEqualStructure(actual, expected)) + { + /// Переименовываем существующую таблицу. + int suffix = 0; + while (context.isTableExist(database_name, table_name + "_" + toString(suffix))) + ++suffix; + + ASTRenameQuery * rename = new ASTRenameQuery; + ASTPtr holder = rename; + + ASTRenameQuery::Table from; + from.database = database_name; + from.table = table_name; + + ASTRenameQuery::Table to; + to.database = database_name; + to.table = table_name + "_" + toString(suffix); + + ASTRenameQuery::Element elem; + elem.from = from; + elem.to = to; + + rename->elements.emplace_back(elem); + + LOG_DEBUG(log, "Existing table " << description << " for query log has obsolete or different structure." + " Renaming it to " << backQuoteIfNeed(to.table)); + + InterpreterRenameQuery(holder, context).execute(); + + /// Нужная таблица будет создана. + table = nullptr; + } + else + LOG_DEBUG(log, "Will use existing table " << description << " for query log."); + } + + if (!table) + { + /// Создаём таблицу. + LOG_DEBUG(log, "Creating new table " << description << " for query log."); + + ASTCreateQuery * create = new ASTCreateQuery; + ASTPtr holder = create; + + create->database = database_name; + create->table = table_name; + + Block sample = createBlock(); + create->columns = InterpreterCreateQuery::formatColumns(sample.getColumnsList()); + + String engine = "MergeTree(event_date, event_time, 8192)"; + ParserFunction engine_parser; + + create->storage = parseQuery(engine_parser, engine.data(), engine.data() + engine.size(), "ENGINE to create table for query log"); + + InterpreterCreateQuery(holder, context).execute(); + + table = context.getTable(database_name, table_name); + } + } + + saving_thread = std::thread([this] { threadFunction(); }); +} + + +QueryLog::~QueryLog() +{ + /// Говорим потоку, что надо завершиться. + QueryLogElement elem; + elem.type = QueryLogElement::SHUTDOWN; + queue.push(elem); + + saving_thread.join(); +} + + +void QueryLog::threadFunction() +{ + Stopwatch time_after_last_write; + bool first = true; + + while (true) + { + try + { + if (first) + { + time_after_last_write.restart(); + first = false; + } + + QueryLogElement element; + bool has_element = false; + + if (data.empty()) + { + queue.pop(element); + has_element = true; + } + else + { + size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000; + if (milliseconds_elapsed < flush_interval_milliseconds) + has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed); + } + + if (has_element) + { + if (element.type == QueryLogElement::SHUTDOWN) + { + flush(); + break; + } + else + data.push_back(element); + } + + size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000; + if (milliseconds_elapsed >= flush_interval_milliseconds) + { + /// Записываем данные в таблицу. + flush(); + time_after_last_write.restart(); + } + } + catch (...) + { + /// В случае ошибки теряем накопленные записи, чтобы не блокироваться. + data.clear(); + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + + +Block QueryLog::createBlock() +{ + return + { + {new ColumnUInt8, new DataTypeUInt8, "type"}, + {new ColumnUInt16, new DataTypeDate, "event_date"}, + {new ColumnUInt32, new DataTypeDateTime, "event_time"}, + {new ColumnUInt32, new DataTypeDateTime, "query_start_time"}, + {new ColumnUInt64, new DataTypeUInt64, "query_duration_ms"}, + + {new ColumnUInt64, new DataTypeUInt64, "read_rows"}, + {new ColumnUInt64, new DataTypeUInt64, "read_bytes"}, + + {new ColumnUInt64, new DataTypeUInt64, "result_rows"}, + {new ColumnUInt64, new DataTypeUInt64, "result_bytes"}, + + {new ColumnUInt64, new DataTypeUInt64, "memory_usage"}, + + {new ColumnString, new DataTypeString, "query"}, + {new ColumnString, new DataTypeString, "exception"}, + {new ColumnString, new DataTypeString, "stack_trace"}, + + {new ColumnUInt8, new DataTypeUInt8, "interface"}, + {new ColumnUInt8, new DataTypeUInt8, "http_method"}, + {new ColumnFixedString(16), new DataTypeFixedString(16), "ip_address"}, + {new ColumnString, new DataTypeString, "user"}, + {new ColumnString, new DataTypeString, "query_id"}, + }; +} + + +void QueryLog::flush() +{ + try + { + LOG_TRACE(log, "Flushing query log"); + + DateLUT & date_lut = DateLUT::instance(); + + Block block = createBlock(); + + for (const QueryLogElement & elem : data) + { + char ipv6_binary[16]; + if (Poco::Net::IPAddress::IPv6 == elem.ip_address.family()) + { + memcpy(ipv6_binary, elem.ip_address.addr(), 16); + } + else if (Poco::Net::IPAddress::IPv4 == elem.ip_address.family()) + { + /// Преобразуем в IPv6-mapped адрес. + memset(ipv6_binary, 0, 10); + ipv6_binary[10] = '\xFF'; + ipv6_binary[11] = '\xFF'; + memcpy(&ipv6_binary[12], elem.ip_address.addr(), 4); + } + else + memset(ipv6_binary, 0, 16); + + size_t i = 0; + + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.type)); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(date_lut.toDayNum(elem.event_time))); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.event_time)); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.query_start_time)); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.query_duration_ms)); + + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.read_rows)); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.read_bytes)); + + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.result_rows)); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.result_bytes)); + + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.memory_usage)); + + block.unsafeGetByPosition(i++).column.get()->insertData(elem.query.data(), elem.query.size()); + block.unsafeGetByPosition(i++).column.get()->insertData(elem.exception.data(), elem.exception.size()); + block.unsafeGetByPosition(i++).column.get()->insertData(elem.stack_trace.data(), elem.stack_trace.size()); + + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.interface)); + block.unsafeGetByPosition(i++).column.get()->insert(static_cast(elem.http_method)); + + block.unsafeGetByPosition(i++).column.get()->insertData(ipv6_binary, 16); + + block.unsafeGetByPosition(i++).column.get()->insertData(elem.user.data(), elem.user.size()); + block.unsafeGetByPosition(i++).column.get()->insertData(elem.query_id.data(), elem.query_id.size()); + } + + BlockOutputStreamPtr stream = table->write(nullptr); + + stream->writePrefix(); + stream->write(block); + stream->writeSuffix(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + /// В случае ошибки тоже очищаем накопленные записи, чтобы не блокироваться. + data.clear(); +} + +} diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index af59ddd2afa..9a253ca96ee 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -66,7 +66,7 @@ void QuotaForInterval::addQuery(time_t current_time, const String & quota_name) __sync_fetch_and_add(&used.queries, 1); } -void QuotaForInterval::addError(time_t current_time, const String & quota_name) +void QuotaForInterval::addError(time_t current_time, const String & quota_name) noexcept { __sync_fetch_and_add(&used.errors, 1); } @@ -181,7 +181,7 @@ void QuotaForIntervals::addQuery(time_t current_time) it->second.addQuery(current_time, name); } -void QuotaForIntervals::addError(time_t current_time) +void QuotaForIntervals::addError(time_t current_time) noexcept { for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) it->second.addError(current_time, name); diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 91fdca2b2ad..44f747da0f2 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -1,10 +1,12 @@ #include +#include #include #include #include #include +#include #include #include @@ -15,6 +17,7 @@ #include #include #include +#include #include @@ -31,6 +34,7 @@ static void checkLimits(const IAST & ast, const Limits & limits) } +/// Логгировать запрос в обычный лог (не в таблицу). static void logQuery(const String & query, const Context & context) { String logged_query = query; @@ -39,6 +43,53 @@ static void logQuery(const String & query, const Context & context) } +static void setClientInfo(QueryLogElement & elem, Context & context) +{ + elem.interface = context.getInterface(); + elem.http_method = context.getHTTPMethod(); + elem.ip_address = context.getIPAddress(); + elem.user = context.getUser(); + elem.query_id = context.getCurrentQueryId(); +} + + +static void onExceptionBeforeStart(const String & query, Context & context, time_t current_time) +{ + /// Эксепшен до начала выполнения запроса. + context.getQuota().addError(current_time); + + bool log_queries = context.getSettingsRef().log_queries; + + /// Логгируем в таблицу начало выполнения запроса, если нужно. + if (log_queries) + { + QueryLogElement elem; + + elem.type = QueryLogElement::EXCEPTION_BEFORE_START; + + elem.event_time = current_time; + elem.query_start_time = current_time; + + elem.query = query; + elem.exception = getCurrentExceptionMessage(false); + + setClientInfo(elem, context); + + try + { + throw; + } + catch (const Exception & e) + { + elem.stack_trace = e.getStackTrace().toString(); + } + catch (...) {} + + context.getQueryLog().add(elem); + } +} + + static std::tuple executeQueryImpl( IParser::Pos begin, IParser::Pos end, @@ -47,6 +98,7 @@ static std::tuple executeQueryImpl( QueryProcessingStage::Enum stage) { ProfileEvents::increment(ProfileEvents::Query); + time_t current_time = time(0); ParserQuery parser; ASTPtr ast; @@ -67,58 +119,176 @@ static std::tuple executeQueryImpl( catch (...) { /// Всё равно логгируем запрос. - logQuery(String(begin, begin + std::min(end - begin, static_cast(max_query_size))), context); + if (!internal) + { + String query = String(begin, begin + std::min(end - begin, static_cast(max_query_size))); + logQuery(query, context); + onExceptionBeforeStart(query, context, current_time); + } throw; } String query(begin, query_size); - - if (!internal) - logQuery(query, context); - - /// Проверка ограничений. - checkLimits(*ast, context.getSettingsRef().limits); - - QuotaForIntervals & quota = context.getQuota(); - time_t current_time = time(0); - - quota.checkExceeded(current_time); - - /// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем. - ProcessList::EntryPtr process_list_entry; - if (!internal && nullptr == typeid_cast(&*ast)) - { - const Settings & settings = context.getSettingsRef(); - - process_list_entry = context.getProcessList().insert( - query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(), - settings.limits.max_memory_usage, - settings.queue_max_wait_ms.totalMilliseconds(), - settings.replace_running_query, - settings.priority); - - context.setProcessListElement(&process_list_entry->get()); - } - BlockIO res; try { + if (!internal) + logQuery(query, context); + + /// Проверка ограничений. + checkLimits(*ast, context.getSettingsRef().limits); + + QuotaForIntervals & quota = context.getQuota(); + + quota.checkExceeded(current_time); + + const Settings & settings = context.getSettingsRef(); + + /// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем. + ProcessList::EntryPtr process_list_entry; + if (!internal && nullptr == typeid_cast(&*ast)) + { + process_list_entry = context.getProcessList().insert( + query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(), + settings.limits.max_memory_usage, + settings.queue_max_wait_ms.totalMilliseconds(), + settings.replace_running_query, + settings.priority); + + context.setProcessListElement(&process_list_entry->get()); + } + auto interpreter = InterpreterFactory::get(ast, context, stage); res = interpreter->execute(); /// Держим элемент списка процессов до конца обработки запроса. res.process_list_entry = process_list_entry; + + quota.addQuery(current_time); + + /// Всё, что связано с логом запросов. + { + QueryLogElement elem; + + elem.type = QueryLogElement::QUERY_START; + + elem.event_time = current_time; + elem.query_start_time = current_time; + + elem.query = query; + + setClientInfo(elem, context); + + bool log_queries = settings.log_queries; + + /// Логгируем в таблицу начало выполнения запроса, если нужно. + if (log_queries) + context.getQueryLog().add(elem); + + /// Также дадим вызывающему коду в дальнейшем логгировать завершение запроса и эксепшен. + res.finish_callback = [elem, &context, log_queries] (IBlockInputStream & stream) mutable + { + ProcessListElement * process_list_elem = context.getProcessListElement(); + + if (!process_list_elem) + return; + + double elapsed_seconds = process_list_elem->watch.elapsedSeconds(); + + elem.type = QueryLogElement::QUERY_FINISH; + + elem.event_time = time(0); + elem.query_duration_ms = elapsed_seconds * 1000; + + elem.read_rows = process_list_elem->progress.rows; + elem.read_bytes = process_list_elem->progress.bytes; + + auto memory_usage = process_list_elem->memory_tracker.getPeak(); + elem.memory_usage = memory_usage > 0 ? memory_usage : 0; + + if (IProfilingBlockInputStream * profiling_stream = dynamic_cast(&stream)) + { + const BlockStreamProfileInfo & info = profiling_stream->getInfo(); + + elem.result_rows = info.rows; + elem.result_bytes = info.bytes; + } + + if (elem.read_rows != 0) + { + LOG_INFO(&Logger::get("executeQuery"), std::fixed << std::setprecision(3) + << "Read " << elem.read_rows << " rows, " + << formatReadableSizeWithBinarySuffix(elem.read_bytes) << " in " << elapsed_seconds << " sec., " + << static_cast(elem.read_rows / elapsed_seconds) << " rows/sec., " + << formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds) << "/sec."); + } + + if (log_queries) + context.getQueryLog().add(elem); + }; + + res.exception_callback = [elem, &context, log_queries, current_time] () mutable + { + context.getQuota().addError(current_time); + + elem.type = QueryLogElement::EXCEPTION_WHILE_PROCESSING; + + elem.event_time = time(0); + elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); + elem.exception = getCurrentExceptionMessage(false); + + ProcessListElement * process_list_elem = context.getProcessListElement(); + + if (process_list_elem) + { + double elapsed_seconds = process_list_elem->watch.elapsedSeconds(); + + elem.query_duration_ms = elapsed_seconds * 1000; + + elem.read_rows = process_list_elem->progress.rows; + elem.read_bytes = process_list_elem->progress.bytes; + + auto memory_usage = process_list_elem->memory_tracker.getPeak(); + elem.memory_usage = memory_usage > 0 ? memory_usage : 0; + } + + /// Достаём стек трейс, если возможно. + try + { + throw; + } + catch (const Exception & e) + { + elem.stack_trace = e.getStackTrace().toString(); + + LOG_ERROR(&Logger::get("executeQuery"), elem.exception << ", Stack trace:\n\n" << elem.stack_trace); + } + catch (...) + { + LOG_ERROR(&Logger::get("executeQuery"), elem.exception); + } + + if (log_queries) + context.getQueryLog().add(elem); + }; + + if (!internal && res.in) + { + std::stringstream log_str; + log_str << "Query pipeline:\n"; + res.in->dumpTree(log_str); + LOG_DEBUG(&Logger::get("executeQuery"), log_str.str()); + } + } } catch (...) { - quota.addError(current_time); + onExceptionBeforeStart(query, context, current_time); throw; } - quota.addQuery(current_time); - return std::make_tuple(ast, res); } @@ -174,51 +344,61 @@ void executeQuery( std::tie(ast, streams) = executeQueryImpl(begin, end, context, internal, stage); - if (streams.out) + try { - const ASTInsertQuery * ast_insert_query = dynamic_cast(ast.get()); + if (streams.out) + { + const ASTInsertQuery * ast_insert_query = dynamic_cast(ast.get()); - if (!ast_insert_query) - throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR); + if (!ast_insert_query) + throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR); - String format = ast_insert_query->format; - if (format.empty()) - format = "Values"; + String format = ast_insert_query->format; + if (format.empty()) + format = "Values"; - /// Данные могут содержаться в распарсенной (ast_insert_query.data) и ещё не распарсенной (istr) части запроса. + /// Данные могут содержаться в распарсенной (ast_insert_query.data) и ещё не распарсенной (istr) части запроса. - ConcatReadBuffer::ReadBuffers buffers; - ReadBuffer buf1(const_cast(ast_insert_query->data), ast_insert_query->data ? ast_insert_query->end - ast_insert_query->data : 0, 0); + ConcatReadBuffer::ReadBuffers buffers; + ReadBuffer buf1(const_cast(ast_insert_query->data), ast_insert_query->data ? ast_insert_query->end - ast_insert_query->data : 0, 0); - if (ast_insert_query->data) - buffers.push_back(&buf1); - buffers.push_back(&istr); + if (ast_insert_query->data) + buffers.push_back(&buf1); + buffers.push_back(&istr); - /** NOTE Нельзя читать из istr до того, как прочтём всё между ast_insert_query.data и ast_insert_query.end. - * - потому что query.data может ссылаться на кусок памяти, использующийся в качестве буфера в istr. - */ + /** NOTE Нельзя читать из istr до того, как прочтём всё между ast_insert_query.data и ast_insert_query.end. + * - потому что query.data может ссылаться на кусок памяти, использующийся в качестве буфера в istr. + */ - ConcatReadBuffer data_istr(buffers); + ConcatReadBuffer data_istr(buffers); - BlockInputStreamPtr in{ - context.getFormatFactory().getInput( - format, data_istr, streams.out_sample, context.getSettings().max_insert_block_size)}; + BlockInputStreamPtr in{ + context.getFormatFactory().getInput( + format, data_istr, streams.out_sample, context.getSettings().max_insert_block_size)}; - copyData(*in, *streams.out); + copyData(*in, *streams.out); + } + + if (streams.in) + { + const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); + + String format_name = ast_query_with_output && (ast_query_with_output->getFormat() != nullptr) + ? typeid_cast(*ast_query_with_output->getFormat()).name + : context.getDefaultFormat(); + + BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, ostr, streams.in_sample); + + copyData(*streams.in, *out); + } + } + catch (...) + { + streams.onException(); + throw; } - if (streams.in) - { - const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); - - String format_name = ast_query_with_output && ast_query_with_output->format - ? typeid_cast(*ast_query_with_output->format).name - : context.getDefaultFormat(); - - BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, ostr, streams.in_sample); - - copyData(*streams.in, *out); - } + streams.onFinish(); } } diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp new file mode 100644 index 00000000000..9c6392b1e0f --- /dev/null +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -0,0 +1,82 @@ +#include +#include + + +namespace DB +{ + + +/** Обновляет SipHash по данным Field */ +class FieldVisitorHash : public StaticVisitor<> +{ +private: + SipHash & hash; +public: + FieldVisitorHash(SipHash & hash) : hash(hash) {} + + void operator() (const Null & x) const + { + UInt8 type = Field::Types::Null; + hash.update(reinterpret_cast(&type), sizeof(type)); + } + + void operator() (const UInt64 & x) const + { + UInt8 type = Field::Types::UInt64; + hash.update(reinterpret_cast(&type), sizeof(type)); + hash.update(reinterpret_cast(&x), sizeof(x)); + } + + void operator() (const Int64 & x) const + { + UInt8 type = Field::Types::Int64; + hash.update(reinterpret_cast(&type), sizeof(type)); + hash.update(reinterpret_cast(&x), sizeof(x)); + } + + void operator() (const Float64 & x) const + { + UInt8 type = Field::Types::Float64; + hash.update(reinterpret_cast(&type), sizeof(type)); + hash.update(reinterpret_cast(&x), sizeof(x)); + } + + void operator() (const String & x) const + { + UInt8 type = Field::Types::String; + hash.update(reinterpret_cast(&type), sizeof(type)); + size_t size = x.size(); + hash.update(reinterpret_cast(&size), sizeof(size)); + hash.update(x.data(), x.size()); + } + + void operator() (const Array & x) const + { + UInt8 type = Field::Types::Array; + hash.update(reinterpret_cast(&type), sizeof(type)); + size_t size = x.size(); + hash.update(reinterpret_cast(&size), sizeof(size)); + + for (const auto & elem : x) + apply_visitor(*this, elem); + } +}; + + +String ASTLiteral::getColumnName() const +{ + /// Отдельный случай для очень больших массивов. Вместо указания всех элементов, будем использовать хэш от содержимого. + if (value.getType() == Field::Types::Array + && value.get().size() > 100) /// 100 - наугад. + { + SipHash hash; + apply_visitor(FieldVisitorHash(hash), value); + UInt64 low, high; + hash.get128(low, high); + return "__array_" + toString(low) + "_" + toString(high); + } + + return apply_visitor(FieldVisitorToString(), value); +} + +} diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index f98ec0fa39b..8e8cc668c50 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -172,5 +172,14 @@ ASTPtr ASTSelectQuery::clone() const return ptr; } + +const IAST * ASTSelectQuery::getFormat() const +{ + const ASTSelectQuery * query = this; + while (!query->next_union_all.isNull()) + query = static_cast(query->next_union_all.get()); + return query->format.get(); +} + }; diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 464763c2165..1209af32578 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -405,6 +405,59 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max } +bool ParserArrayOfLiterals::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) +{ + Pos begin = pos; + Array arr; + + if (pos == end || *pos != '[') + { + expected = "opening square bracket"; + return false; + } + + ParserWhiteSpaceOrComments ws; + ParserLiteral literal_p; + + ++pos; + + while (pos != end) + { + ws.ignore(pos, end); + + if (!arr.empty()) + { + if (*pos == ']') + { + ++pos; + node = new ASTLiteral(StringRange(begin, pos), arr); + return true; + } + else if (*pos == ',') + { + ++pos; + } + else + { + expected = "comma or closing square bracket"; + return false; + } + } + + ws.ignore(pos, end); + + ASTPtr literal_node; + if (!literal_p.parse(pos, end, literal_node, max_parsed_pos, expected)) + return false; + + arr.push_back(typeid_cast(*literal_node).value); + } + + expected = "closing square bracket"; + return false; +} + + bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) { ParserNull null_p; @@ -450,6 +503,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ParserParenthesisExpression paren_p; ParserSubquery subquery_p; ParserArray array_p; + ParserArrayOfLiterals array_lite_p; ParserLiteral lit_p; ParserFunction fun_p; ParserCompoundIdentifier id_p; @@ -461,6 +515,9 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & if (paren_p.parse(pos, end, node, max_parsed_pos, expected)) return true; + if (array_lite_p.parse(pos, end, node, max_parsed_pos, expected)) + return true; + if (array_p.parse(pos, end, node, max_parsed_pos, expected)) return true; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 91e806ba436..8a70bdc4a53 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -8,7 +8,6 @@ #include #include - namespace DB { @@ -296,6 +295,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p ws.ignore(pos, end); } + bool has_format = false; + /// FORMAT format_name if (s_format.ignore(pos, end, max_parsed_pos, expected)) { @@ -308,6 +309,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p typeid_cast(*select_query->format).kind = ASTIdentifier::Format; ws.ignore(pos, end); + has_format = true; } // UNION ALL select query @@ -317,6 +319,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p if (s_all.ignore(pos, end, max_parsed_pos, expected)) { + if (has_format) + { + /// FORMAT может быть задан только в последнем запросе цепочки UNION ALL. + expected = "FORMAT only in the last SELECT of the UNION ALL chain"; + return false; + } + ParserSelectQuery select_p; if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected)) return false; diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index cd13d00c57c..7649e3e01ea 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -136,41 +136,24 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net if (readonly) context.getSettingsRef().limits.readonly = true; - Stopwatch watch; + context.setInterface(Context::Interface::HTTP); + + Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = Context::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = Context::HTTPMethod::POST; + + context.setHTTPMethod(http_method); + executeQuery(*in, *used_output.out_maybe_compressed, context, query_plan); - watch.stop(); - - if (query_plan) - { - std::stringstream log_str; - log_str << "Query pipeline:\n"; - query_plan->dumpTree(log_str); - LOG_DEBUG(log, log_str.str()); - - /// Выведем информацию о том, сколько считано строк и байт. - size_t rows = 0; - size_t bytes = 0; - - query_plan->getLeafRowsBytes(rows, bytes); - - if (rows != 0) - { - LOG_INFO(log, std::fixed << std::setprecision(3) - << "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., " - << static_cast(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec."); - } - } - - QuotaForIntervals & quota = context.getQuota(); - if (!quota.empty()) - LOG_INFO(log, "Quota:\n" << quota.toString()); /// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200. used_output.out->finalize(); } -void HTTPHandler::trySendExceptionToClient(std::stringstream & s, +void HTTPHandler::trySendExceptionToClient(const std::string & s, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, Output & used_output) { @@ -191,7 +174,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s, if (!response.sent() && !used_output.out_maybe_compressed) { /// Ещё ничего не отправляли, и даже не знаем, нужно ли сжимать ответ. - response.send() << s.str() << std::endl; + response.send() << s << std::endl; } else if (used_output.out_maybe_compressed) { @@ -207,8 +190,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s, used_output.out->position() = used_output.out->buffer().begin(); } - std::string exception_message = s.str(); - writeString(exception_message, *used_output.out_maybe_compressed); + writeString(s, *used_output.out_maybe_compressed); writeChar('\n', *used_output.out_maybe_compressed); used_output.out_maybe_compressed->next(); used_output.out->finalize(); @@ -245,35 +227,10 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne processQuery(request, response, used_output); LOG_INFO(log, "Done processing query"); } - catch (Exception & e) - { - std::stringstream s; - s << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() - << ", Stack trace:\n\n" << e.getStackTrace().toString(); - LOG_ERROR(log, s.str()); - trySendExceptionToClient(s, request, response, used_output); - } - catch (Poco::Exception & e) - { - std::stringstream s; - s << "Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() - << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); - LOG_ERROR(log, s.str()); - trySendExceptionToClient(s, request, response, used_output); - } - catch (std::exception & e) - { - std::stringstream s; - s << "Code: " << ErrorCodes::STD_EXCEPTION << ". " << e.what(); - LOG_ERROR(log, s.str()); - trySendExceptionToClient(s, request, response, used_output); - } catch (...) { - std::stringstream s; - s << "Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ". Unknown exception."; - LOG_ERROR(log, s.str()); - trySendExceptionToClient(s, request, response, used_output); + tryLogCurrentException(log); + trySendExceptionToClient(getCurrentExceptionMessage(true), request, response, used_output); } } diff --git a/dbms/src/Server/HTTPHandler.h b/dbms/src/Server/HTTPHandler.h index 7357c7f5831..ae8a2c5c564 100644 --- a/dbms/src/Server/HTTPHandler.h +++ b/dbms/src/Server/HTTPHandler.h @@ -26,7 +26,7 @@ public: void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response); - void trySendExceptionToClient(std::stringstream & s, + void trySendExceptionToClient(const std::string & s, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, Output & used_output); diff --git a/dbms/src/Server/OLAPHTTPHandler.cpp b/dbms/src/Server/OLAPHTTPHandler.cpp index 7a2bbe24485..dc61f6920fa 100644 --- a/dbms/src/Server/OLAPHTTPHandler.cpp +++ b/dbms/src/Server/OLAPHTTPHandler.cpp @@ -52,6 +52,9 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco: context.setUser(user, password, request.clientAddress().host(), quota_key); + context.setInterface(Context::Interface::HTTP); + context.setHTTPMethod(Context::HTTPMethod::POST); + OLAP::QueryParseResult olap_query = server.olap_parser->parse(request_istream); std::string clickhouse_query; @@ -66,31 +69,6 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco: executeQuery(in, out, context, query_plan); watch.stop(); - if (query_plan) - { - std::stringstream log_str; - log_str << "Query pipeline:\n"; - query_plan->dumpTree(log_str); - LOG_DEBUG(log, log_str.str()); - - /// Выведем информацию о том, сколько считано строк и байт. - size_t rows = 0; - size_t bytes = 0; - - query_plan->getLeafRowsBytes(rows, bytes); - - if (rows != 0) - { - LOG_INFO(log, std::fixed << std::setprecision(3) - << "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., " - << static_cast(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec."); - } - } - - QuotaForIntervals & quota = context.getQuota(); - if (!quota.empty()) - LOG_INFO(log, "Quota:\n" << quota.toString()); - /// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200. out.finalize(); } diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 2188ae8a15c..3c2e42a5ccb 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -533,7 +533,12 @@ int Server::main(const std::vector & args) LOG_DEBUG(log, "Loaded metadata."); /// Создаём системные таблицы. - global_context->addDatabase("system"); + if (!global_context->isDatabaseExist("system")) + { + Poco::File(path + "data/system").createDirectories(); + Poco::File(path + "metadata/system").createDirectories(); + global_context->addDatabase("system"); + } global_context->addTable("system", "one", StorageSystemOne::create("one")); global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers")); @@ -546,7 +551,7 @@ int Server::main(const std::vector & args) global_context->addTable("system", "events", StorageSystemEvents::create("events")); global_context->addTable("system", "merges", StorageSystemMerges::create("merges")); global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas")); - global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries")); + global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries")); global_context->addTable("system", "columns", StorageSystemColumns::create("columns")); global_context->addTable("system", "functions", StorageSystemFunctions::create("functions")); global_context->addTable("system", "clusters", StorageSystemClusters::create("clusters", *global_context)); diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 395fb769baa..8c9231c94f1 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -111,6 +111,7 @@ void TCPHandler::runImpl() { /// Восстанавливаем контекст запроса. query_context = connection_context; + query_context.setInterface(Context::Interface::TCP); /** Если Query - обрабатываем. Если Ping или Cancel - возвращаемся в начало. * Могут прийти настройки на отдельный запрос, которые модифицируют query_context. @@ -148,8 +149,7 @@ void TCPHandler::runImpl() } catch (const Exception & e) { - LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() - << ", Stack trace:\n\n" << e.getStackTrace().toString()); + state.io.onException(); exception = e.clone(); if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT) @@ -163,24 +163,22 @@ void TCPHandler::runImpl() * Хотя в одном из них, мы должны отправить эксепшен клиенту, а в другом - не можем. * Будем пытаться отправить эксепшен клиенту в любом случае - см. ниже. */ - LOG_ERROR(log, "Poco::Net::NetException. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() - << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()); + state.io.onException(); exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION); } catch (const Poco::Exception & e) { - LOG_ERROR(log, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() - << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()); + state.io.onException(); exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION); } catch (const std::exception & e) { - LOG_ERROR(log, "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what()); + state.io.onException(); exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION); } catch (...) { - LOG_ERROR(log, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION); + state.io.onException(); exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION); } @@ -285,11 +283,6 @@ void TCPHandler::processOrdinaryQuery() AsynchronousBlockInputStream async_in(state.io.in); async_in.readPrefix(); - std::stringstream query_pipeline; - async_in.dumpTree(query_pipeline); - LOG_DEBUG(log, "Query pipeline:\n" << query_pipeline.rdbuf()); - - Stopwatch watch; while (true) { Block block; @@ -341,9 +334,7 @@ void TCPHandler::processOrdinaryQuery() } async_in.readSuffix(); - - watch.stop(); - logProfileInfo(watch, *state.io.in); + state.io.onFinish(); } } @@ -412,27 +403,6 @@ void TCPHandler::sendExtremes() } -void TCPHandler::logProfileInfo(Stopwatch & watch, IBlockInputStream & in) -{ - /// Выведем информацию о том, сколько считано строк и байт. - size_t rows = 0; - size_t bytes = 0; - - in.getLeafRowsBytes(rows, bytes); - - if (rows != 0) - { - LOG_INFO(log, std::fixed << std::setprecision(3) - << "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., " - << static_cast(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec."); - } - - QuotaForIntervals & quota = query_context.getQuota(); - if (!quota.empty()) - LOG_INFO(log, "Quota:\n" << quota.toString()); -} - - void TCPHandler::receiveHello() { /// Получить hello пакет. diff --git a/dbms/src/Server/TCPHandler.h b/dbms/src/Server/TCPHandler.h index ffae284d459..1b45cf72191 100644 --- a/dbms/src/Server/TCPHandler.h +++ b/dbms/src/Server/TCPHandler.h @@ -130,9 +130,6 @@ private: /// Эта функция вызывается из разных потоков. void updateProgress(const Progress & value); - - /// Вывести информацию о скорости выполнения SELECT запроса. - void logProfileInfo(Stopwatch & watch, IBlockInputStream & in); }; diff --git a/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp b/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp index 3bd49d17231..c8e8a84f3fe 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp @@ -11,7 +11,6 @@ #include #include #include -#include namespace DB @@ -116,12 +115,6 @@ struct Stream readIntBinary(mrk_mark.offset_in_compressed_file, mrk_hashing_buf); readIntBinary(mrk_mark.offset_in_decompressed_block, mrk_hashing_buf); - /// На всякий случай, сохраним смещение в файле и размер предыдущего блока. - SCOPE_EXIT( - prev_offset_in_compressed_file = mrk_mark.offset_in_compressed_file; - prev_buffer_size = uncompressed_hashing_buf.buffer().size(); - ); - bool has_alternative_mark = false; MarkInCompressedFile alternative_data_mark; MarkInCompressedFile data_mark; @@ -145,18 +138,6 @@ struct Stream if (uncompressed_hashing_buf.eof()) return; } - else if (uncompressed_hashing_buf.offset() == 0) - { - /// Восстановим засечку на конец предыдущего блока по сохраненным данным - has_alternative_mark = true; - alternative_data_mark.offset_in_compressed_file = prev_offset_in_compressed_file; - alternative_data_mark.offset_in_decompressed_block = prev_buffer_size; - - if (mrk_mark == alternative_data_mark) - return; - } - - std::cout << "mrk_mark " << mrk_mark.offset_in_compressed_file << ' ' << mrk_mark.offset_in_decompressed_block << std::endl; data_mark.offset_in_compressed_file = compressed_hashing_buf.count() - uncompressing_buf.getSizeCompressed(); data_mark.offset_in_decompressed_block = uncompressed_hashing_buf.offset(); @@ -180,10 +161,6 @@ struct Stream checksums.files[name + ".mrk"] = MergeTreeData::DataPart::Checksums::Checksum( mrk_hashing_buf.count(), mrk_hashing_buf.getHash()); } - -private: - size_t prev_offset_in_compressed_file{}; - size_t prev_buffer_size{}; }; /// Возвращает количество строк. Добавляет в checksums чексуммы всех файлов столбца. diff --git a/dbms/src/Storages/MergeTree/PKCondition.cpp b/dbms/src/Storages/MergeTree/PKCondition.cpp index ab680230d44..445a47b3f81 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.cpp +++ b/dbms/src/Storages/MergeTree/PKCondition.cpp @@ -8,6 +8,136 @@ namespace DB { + +/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем. +UInt64 stringToDateOrDateTime(const String & s) +{ + ReadBufferFromString in(s); + + if (s.size() == strlen("YYYY-MM-DD")) + { + DayNum_t date{}; + readDateText(date, in); + return UInt64(date); + } + else + { + time_t date_time{}; + readDateTimeText(date_time, in); + if (!in.eof()) + throw Exception("String is too long for DateTime: " + s); + return UInt64(date_time); + } +} + + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" + +/** Более точное сравнение. + * Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой. + * Правила сравнения - такие же, что и в FunctionsComparison. + * В том числе, сравнение знаковых и беззнаковых оставляем UB. + */ +class FieldVisitorAccurateEquals : public StaticVisitor +{ +public: + bool operator() (const Null & l, const Null & r) const { return true; } + bool operator() (const Null & l, const UInt64 & r) const { return false; } + bool operator() (const Null & l, const Int64 & r) const { return false; } + bool operator() (const Null & l, const Float64 & r) const { return false; } + bool operator() (const Null & l, const String & r) const { return false; } + bool operator() (const Null & l, const Array & r) const { return false; } + + bool operator() (const UInt64 & l, const Null & r) const { return false; } + bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; } + bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; } + bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; } + bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); } + bool operator() (const UInt64 & l, const Array & r) const { return false; } + + bool operator() (const Int64 & l, const Null & r) const { return false; } + bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; } + bool operator() (const Int64 & l, const Int64 & r) const { return l == r; } + bool operator() (const Int64 & l, const Float64 & r) const { return l == r; } + bool operator() (const Int64 & l, const String & r) const { return false; } + bool operator() (const Int64 & l, const Array & r) const { return false; } + + bool operator() (const Float64 & l, const Null & r) const { return false; } + bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; } + bool operator() (const Float64 & l, const Int64 & r) const { return l == r; } + bool operator() (const Float64 & l, const Float64 & r) const { return l == r; } + bool operator() (const Float64 & l, const String & r) const { return false; } + bool operator() (const Float64 & l, const Array & r) const { return false; } + + bool operator() (const String & l, const Null & r) const { return false; } + bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; } + bool operator() (const String & l, const Int64 & r) const { return false; } + bool operator() (const String & l, const Float64 & r) const { return false; } + bool operator() (const String & l, const String & r) const { return l == r; } + bool operator() (const String & l, const Array & r) const { return false; } + + bool operator() (const Array & l, const Null & r) const { return false; } + bool operator() (const Array & l, const UInt64 & r) const { return false; } + bool operator() (const Array & l, const Int64 & r) const { return false; } + bool operator() (const Array & l, const Float64 & r) const { return false; } + bool operator() (const Array & l, const String & r) const { return false; } + bool operator() (const Array & l, const Array & r) const { return l == r; } +}; + +class FieldVisitorAccurateLess : public StaticVisitor +{ +public: + bool operator() (const Null & l, const Null & r) const { return false; } + bool operator() (const Null & l, const UInt64 & r) const { return true; } + bool operator() (const Null & l, const Int64 & r) const { return true; } + bool operator() (const Null & l, const Float64 & r) const { return true; } + bool operator() (const Null & l, const String & r) const { return true; } + bool operator() (const Null & l, const Array & r) const { return true; } + + bool operator() (const UInt64 & l, const Null & r) const { return false; } + bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; } + bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; } + bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; } + bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); } + bool operator() (const UInt64 & l, const Array & r) const { return true; } + + bool operator() (const Int64 & l, const Null & r) const { return false; } + bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; } + bool operator() (const Int64 & l, const Int64 & r) const { return l < r; } + bool operator() (const Int64 & l, const Float64 & r) const { return l < r; } + bool operator() (const Int64 & l, const String & r) const { return true; } + bool operator() (const Int64 & l, const Array & r) const { return true; } + + bool operator() (const Float64 & l, const Null & r) const { return false; } + bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; } + bool operator() (const Float64 & l, const Int64 & r) const { return l < r; } + bool operator() (const Float64 & l, const Float64 & r) const { return l < r; } + bool operator() (const Float64 & l, const String & r) const { return true; } + bool operator() (const Float64 & l, const Array & r) const { return true; } + + bool operator() (const String & l, const Null & r) const { return false; } + bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; } + bool operator() (const String & l, const Int64 & r) const { return false; } + bool operator() (const String & l, const Float64 & r) const { return false; } + bool operator() (const String & l, const String & r) const { return l < r; } + bool operator() (const String & l, const Array & r) const { return true; } + + bool operator() (const Array & l, const Null & r) const { return false; } + bool operator() (const Array & l, const UInt64 & r) const { return false; } + bool operator() (const Array & l, const Int64 & r) const { return false; } + bool operator() (const Array & l, const Float64 & r) const { return false; } + bool operator() (const Array & l, const String & r) const { return false; } + bool operator() (const Array & l, const Array & r) const { return l < r; } +}; + +#pragma GCC diagnostic pop + + +inline bool Range::equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); } +inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); } + + PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_) : sort_descr(sort_descr_) { diff --git a/dbms/src/Storages/StorageChunkMerger.cpp b/dbms/src/Storages/StorageChunkMerger.cpp index 964d4af9a7e..e314f127b08 100644 --- a/dbms/src/Storages/StorageChunkMerger.cpp +++ b/dbms/src/Storages/StorageChunkMerger.cpp @@ -545,7 +545,8 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks) { LOG_INFO(log, "Shutdown requested while merging chunks."); output->writeSuffix(); - new_storage.removeReference(); /// После этого временные данные удалятся. + output = nullptr; + executeQuery("DROP TABLE IF EXISTS " + new_table_full_name, context, true); return false; } @@ -575,14 +576,16 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks) /// Отцепляем исходную таблицу. Ее данные и метаданные остаются на диске. tables_to_drop.push_back(context.detachTable(chunk_merger.source_database, src_name)); - /// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни, ни метаданных. + /// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни данных, ни метаданных. try { - context.addTable(chunk_merger.source_database, src_name, StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false)); + context.addTable(chunk_merger.source_database, src_name, + StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false)); } catch (...) { - LOG_ERROR(log, "Chunk " + src_name + " was removed but not replaced. Its data is stored in table " << new_table_name << ". You may need to resolve this manually."); + LOG_ERROR(log, "Chunk " + src_name + " was removed but not replaced. Its data is stored in table " + << new_table_name << ". You may need to resolve this manually."); throw; } @@ -601,9 +604,6 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks) /// что-нибудь может сломаться. } - /// Сейчас на new_storage ссылаются таблицы типа ChunkRef. Удалим лишнюю ссылку, которая была при создании. - new_storage.removeReference(); - LOG_TRACE(log, "Merged chunks."); return true; @@ -613,6 +613,7 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks) Poco::ScopedLock lock(context.getMutex()); currently_written_groups.erase(new_table_full_name); + executeQuery("DROP TABLE IF EXISTS " + new_table_full_name, context, true); throw; } diff --git a/dbms/src/Storages/StorageChunks.cpp b/dbms/src/Storages/StorageChunks.cpp index 68ec39e358d..aa55c1b43c0 100644 --- a/dbms/src/Storages/StorageChunks.cpp +++ b/dbms/src/Storages/StorageChunks.cpp @@ -161,9 +161,6 @@ StorageChunks::StorageChunks( context(context_), log(&Logger::get("StorageChunks")) { - if (!attach) - reference_counter.add(1, true); - _table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table"); try diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index a807f8d1f69..700f4c8a766 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -154,7 +154,7 @@ BlockInputStreams StorageDistributed::read( size_t result_size = (cluster.pools.size() * settings.max_parallel_replicas) + cluster.getLocalNodesNum(); - processed_stage = result_size == 1 + processed_stage = result_size == 1 || settings.distributed_group_by_no_merge ? QueryProcessingStage::Complete : QueryProcessingStage::WithMergeableState; diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 8527fd32f54..f4bc9b587ae 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -233,7 +233,15 @@ Block LogBlockInputStream::readImpl() else column.column = column.type->createColumn(); - readData(*it, *column.type, *column.column, max_rows_to_read, 0, read_offsets); + try + { + readData(*it, *column.type, *column.column, max_rows_to_read, 0, read_offsets); + } + catch (Exception & e) + { + e.addMessage("while reading column " + *it + " at " + storage.path + escapeForFileName(storage.name)); + throw; + } if (column.column->size()) res.insert(column); diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index c8c357ddad5..acde326fb58 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -198,7 +198,15 @@ Block TinyLogBlockInputStream::readImpl() else column.column = column.type->createColumn(); - readData(*it, *column.type, *column.column, block_size, 0, read_offsets); + try + { + readData(*it, *column.type, *column.column, block_size, 0, read_offsets); + } + catch (Exception & e) + { + e.addMessage("while reading column " + *it + " at " + storage.full_path()); + throw; + } if (column.column->size()) res.insert(column); diff --git a/dbms/tests/queries/0_stateless/00098_k_union_all.reference b/dbms/tests/queries/0_stateless/00098_k_union_all.reference new file mode 100644 index 00000000000..1b21a751afc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00098_k_union_all.reference @@ -0,0 +1,18 @@ +1 + +1 +1 + +1 +1 + +1 +1 + +1 +1 + +1 +1 + +1 diff --git a/dbms/tests/queries/0_stateless/00098_k_union_all.sql b/dbms/tests/queries/0_stateless/00098_k_union_all.sql new file mode 100644 index 00000000000..311e5bb19c4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00098_k_union_all.sql @@ -0,0 +1,3 @@ +SELECT 1 FORMAT PrettySpace; +SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace; +SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace; diff --git a/dbms/tests/queries/0_stateless/00180_attach_materialized_view.reference b/dbms/tests/queries/0_stateless/00180_attach_materialized_view.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql b/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql new file mode 100644 index 00000000000..81b162ee4dd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00180_attach_materialized_view.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test.t; +DROP TABLE IF EXISTS test.mv; +DROP TABLE IF EXISTS test.`.inner.mv`; + +CREATE TABLE test.t (x UInt8) ENGINE = Null; +CREATE MATERIALIZED VIEW test.mv AS SELECT * FROM test.t; + +DETACH TABLE test.mv; +ATTACH MATERIALIZED VIEW test.mv AS SELECT * FROM test.t; + +DROP TABLE test.t; +DROP TABLE test.mv; diff --git a/dbms/tests/queries/0_stateless/00181_aggregate_functions_statistics.reference b/dbms/tests/queries/0_stateless/00181_aggregate_functions_statistics.reference new file mode 100644 index 00000000000..ce706690f1a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00181_aggregate_functions_statistics.reference @@ -0,0 +1,14 @@ +inf +0 +inf +0 +0 +0 +0 +0 +inf +0 +0 +0 +inf +0 diff --git a/dbms/tests/queries/0_stateless/00181_aggregate_functions_statistics.sql b/dbms/tests/queries/0_stateless/00181_aggregate_functions_statistics.sql new file mode 100644 index 00000000000..344b02b2d98 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00181_aggregate_functions_statistics.sql @@ -0,0 +1,143 @@ +DROP TABLE IF EXISTS series; + +CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory; + +/* Тестовые данные */ + +INSERT INTO series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3); + +/* varSamp */ + +SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0); +SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1); + +SELECT round(abs(res1 - res2), 6) FROM +( +SELECT + varSamp(x_value) AS res1, + (sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1) AS res2 +FROM series +); + +/* stddevSamp */ + +SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0); +SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1); + +SELECT round(abs(res1 - res2), 6) FROM +( +SELECT + stddevSamp(x_value) AS res1, + sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1)) AS res2 +FROM series +); + +/* varPop */ + +SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 0); +SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 1); + +SELECT round(abs(res1 - res2), 6) FROM +( +SELECT + varPop(x_value) AS res1, + (sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count() AS res2 +FROM series +); + +/* stddevPop */ + +SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 0); +SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 1); + +SELECT round(abs(res1 - res2), 6) FROM +( +SELECT + stddevPop(x_value) AS res1, + sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count()) AS res2 +FROM series +); + +/* covarSamp */ + +SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0); +SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1); + +SELECT round(abs(COVAR1 - COVAR2), 6) +FROM +( + SELECT + arrayJoin([1]) AS ID2, + covarSamp(x_value, y_value) AS COVAR1 + FROM series +) ANY INNER JOIN +( + SELECT + arrayJoin([1]) AS ID2, + sum(VAL) / (count() - 1) AS COVAR2 + FROM + ( + SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL + FROM + ( + SELECT + toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID, + avg(x_value) AS AVG_X, + avg(y_value) AS AVG_Y + FROM series + ) ANY INNER JOIN + ( + SELECT + i AS ID, + x_value AS X, + y_value AS Y + FROM series + ) USING ID + ) +) USING ID2; + +/* covarPop */ + +SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0); +SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1); + +SELECT round(abs(COVAR1 - COVAR2), 6) +FROM +( + SELECT + arrayJoin([1]) AS ID2, + covarPop(x_value, y_value) AS COVAR1 + FROM series +) ANY INNER JOIN +( + SELECT + arrayJoin([1]) AS ID2, + sum(VAL) / count() AS COVAR2 + FROM + ( + SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL + FROM + ( + SELECT + toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID, + avg(x_value) AS AVG_X, + avg(y_value) AS AVG_Y + FROM series + ) ANY INNER JOIN + ( + SELECT + i AS ID, + x_value AS X, + y_value AS Y + FROM series + ) USING ID + ) +) USING ID2; + +/* corr */ + +SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0); +SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1); + +SELECT round(abs(corr(x_value, y_value) - covarPop(x_value, y_value) / (stddevPop(x_value) * stddevPop(y_value))), 6) FROM series; + diff --git a/dbms/tests/queries/0_stateless/00182_functions_higher_order_and_consts.reference b/dbms/tests/queries/0_stateless/00182_functions_higher_order_and_consts.reference new file mode 100644 index 00000000000..4aa1757a136 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00182_functions_higher_order_and_consts.reference @@ -0,0 +1,914 @@ +---map-- +[] +[123,123,123] +[] +[123] +[123,123] +[123,123,123] +[123,123,123,123] +[123,123,123,123,123] +[123,123,123,123,123,123] +[123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123,123] +---filter-- +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[1,2,3] +[] +[0] +[0,1] +[0,1,2] +[0,1,2,3] +[0,1,2,3,4] +[0,1,2,3,4,5] +[0,1,2,3,4,5,6] +[0,1,2,3,4,5,6,7] +[0,1,2,3,4,5,6,7,8] +---count--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +3 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +---sum--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +30 +0 +10 +20 +30 +40 +50 +60 +70 +80 +90 +---all--- +1 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---exists--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---first--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +---first index--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---map-- +[] +[123,123,123] +[] +[123] +[123,123] +[123,123,123] +[123,123,123,123] +[123,123,123,123,123] +[123,123,123,123,123,123] +[123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123,123] +---filter-- +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[1,2,3] +[] +[0] +[0,1] +[0,1,2] +[0,1,2,3] +[0,1,2,3,4] +[0,1,2,3,4,5] +[0,1,2,3,4,5,6] +[0,1,2,3,4,5,6,7] +[0,1,2,3,4,5,6,7,8] +---count--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +3 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +---sum--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +30 +0 +10 +20 +30 +40 +50 +60 +70 +80 +90 +---all--- +1 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---exists--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---first--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +---first index--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---map-- +[] +[123,123,123] +[] +[123] +[123,123] +[123,123,123] +[123,123,123,123] +[123,123,123,123,123] +[123,123,123,123,123,123] +[123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123,123] +---filter-- +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +['1','2','3'] +[] +['0'] +['0','1'] +['0','1','2'] +['0','1','2','3'] +['0','1','2','3','4'] +['0','1','2','3','4','5'] +['0','1','2','3','4','5','6'] +['0','1','2','3','4','5','6','7'] +['0','1','2','3','4','5','6','7','8'] +---count--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +3 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +---sum--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +30 +0 +10 +20 +30 +40 +50 +60 +70 +80 +90 +---all--- +1 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---exists--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---first--- + + + + + + + + + + + + + +1 + +0 +0 +0 +0 +0 +0 +0 +0 +0 +---first index--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---map-- +[] +[123,123,123] +[] +[123] +[123,123] +[123,123,123] +[123,123,123,123] +[123,123,123,123,123] +[123,123,123,123,123,123] +[123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123] +[123,123,123,123,123,123,123,123,123] +---filter-- +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +[] +['1','2','3'] +[] +['0'] +['0','1'] +['0','1','2'] +['0','1','2','3'] +['0','1','2','3','4'] +['0','1','2','3','4','5'] +['0','1','2','3','4','5','6'] +['0','1','2','3','4','5','6','7'] +['0','1','2','3','4','5','6','7','8'] +---count--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +3 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +---sum--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +30 +0 +10 +20 +30 +40 +50 +60 +70 +80 +90 +---all--- +1 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---exists--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +---first--- + + + + + + + + + + + + + +1 + +0 +0 +0 +0 +0 +0 +0 +0 +0 +---first index--- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +--- --- +[] +[1] +[0,0] +[1,1,1] +[0,0,0,0] +[1,1,1,1,1] +[0,0,0,0,0,0] +[1,1,1,1,1,1,1] +[0,0,0,0,0,0,0,0] +[1,1,1,1,1,1,1,1,1] +[] +[0] +[] +[0,1,2] +[] +[0,1,2,3,4] +[] +[0,1,2,3,4,5,6] +[] +[0,1,2,3,4,5,6,7,8] +0 +1 +0 +3 +0 +5 +0 +7 +0 +9 +0 +1 +0 +3 +0 +5 +0 +7 +0 +9 +1 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +--- --- +[] +[1] +[0,0] +[1,1,1] +[0,0,0,0] +[1,1,1,1,1] +[0,0,0,0,0,0] +[1,1,1,1,1,1,1] +[0,0,0,0,0,0,0,0] +[1,1,1,1,1,1,1,1,1] +[] +['0'] +[] +['0','1','2'] +[] +['0','1','2','3','4'] +[] +['0','1','2','3','4','5','6'] +[] +['0','1','2','3','4','5','6','7','8'] +0 +1 +0 +3 +0 +5 +0 +7 +0 +9 +0 +1 +0 +3 +0 +5 +0 +7 +0 +9 +1 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 + +0 + +0 + +0 + +0 + +0 +0 +1 +0 +1 +0 +1 +0 +1 +0 +1 diff --git a/dbms/tests/queries/0_stateless/00182_functions_higher_order_and_consts.sql b/dbms/tests/queries/0_stateless/00182_functions_higher_order_and_consts.sql new file mode 100644 index 00000000000..5513df2e1b0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00182_functions_higher_order_and_consts.sql @@ -0,0 +1,235 @@ +SELECT '---map--'; +SELECT arrayMap(x -> 123, emptyArrayUInt8()); +SELECT arrayMap(x -> 123, [1, 2, 3]); +SELECT arrayMap(x -> 123, range(number)) FROM system.numbers LIMIT 10; +SELECT '---filter--'; +SELECT arrayFilter(x -> 0, emptyArrayUInt8()); +SELECT arrayFilter(x -> 0, [1, 2, 3]); +SELECT arrayFilter(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFilter(x -> 1, emptyArrayUInt8()); +SELECT arrayFilter(x -> 1, [1, 2, 3]); +SELECT arrayFilter(x -> 1, range(number)) FROM system.numbers LIMIT 10; +SELECT '---count---'; +SELECT arrayCount(x -> 0, emptyArrayUInt8()); +SELECT arrayCount(x -> 0, [1, 2, 3]); +SELECT arrayCount(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayCount(x -> 1, emptyArrayUInt8()); +SELECT arrayCount(x -> 1, [1, 2, 3]); +SELECT arrayCount(x -> 1, range(number)) FROM system.numbers LIMIT 10; +SELECT '---sum---'; +SELECT arraySum(x -> 0, emptyArrayUInt8()); +SELECT arraySum(x -> 0, [1, 2, 3]); +SELECT arraySum(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arraySum(x -> 10, emptyArrayUInt8()); +SELECT arraySum(x -> 10, [1, 2, 3]); +SELECT arraySum(x -> 10, range(number)) FROM system.numbers LIMIT 10; +SELECT '---all---'; +SELECT arrayAll(x -> 0, emptyArrayUInt8()); +SELECT arrayAll(x -> 0, [1, 2, 3]); +SELECT arrayAll(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayAll(x -> 1, emptyArrayUInt8()); +SELECT arrayAll(x -> 1, [1, 2, 3]); +SELECT arrayAll(x -> 1, range(number)) FROM system.numbers LIMIT 10; +SELECT '---exists---'; +SELECT arrayExists(x -> 0, emptyArrayUInt8()); +SELECT arrayExists(x -> 0, [1, 2, 3]); +SELECT arrayExists(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayExists(x -> 1, emptyArrayUInt8()); +SELECT arrayExists(x -> 1, [1, 2, 3]); +SELECT arrayExists(x -> 1, range(number)) FROM system.numbers LIMIT 10; +SELECT '---first---'; +SELECT arrayFirst(x -> 0, emptyArrayUInt8()); +SELECT arrayFirst(x -> 0, [1, 2, 3]); +SELECT arrayFirst(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFirst(x -> 1, emptyArrayUInt8()); +SELECT arrayFirst(x -> 1, [1, 2, 3]); +SELECT arrayFirst(x -> 1, range(number)) FROM system.numbers LIMIT 10; +SELECT '---first index---'; +SELECT arrayFirstIndex(x -> 0, emptyArrayUInt8()); +SELECT arrayFirstIndex(x -> 0, [1, 2, 3]); +SELECT arrayFirstIndex(x -> 0, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFirstIndex(x -> 1, emptyArrayUInt8()); +SELECT arrayFirstIndex(x -> 1, [1, 2, 3]); +SELECT arrayFirstIndex(x -> 1, range(number)) FROM system.numbers LIMIT 10; + +SELECT '---map--'; +SELECT arrayMap(x -> materialize(123), emptyArrayUInt8()); +SELECT arrayMap(x -> materialize(123), [1, 2, 3]); +SELECT arrayMap(x -> materialize(123), range(number)) FROM system.numbers LIMIT 10; +SELECT '---filter--'; +SELECT arrayFilter(x -> materialize(0), emptyArrayUInt8()); +SELECT arrayFilter(x -> materialize(0), [1, 2, 3]); +SELECT arrayFilter(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFilter(x -> materialize(1), emptyArrayUInt8()); +SELECT arrayFilter(x -> materialize(1), [1, 2, 3]); +SELECT arrayFilter(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10; +SELECT '---count---'; +SELECT arrayCount(x -> materialize(0), emptyArrayUInt8()); +SELECT arrayCount(x -> materialize(0), [1, 2, 3]); +SELECT arrayCount(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arrayCount(x -> materialize(1), emptyArrayUInt8()); +SELECT arrayCount(x -> materialize(1), [1, 2, 3]); +SELECT arrayCount(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10; +SELECT '---sum---'; +SELECT arraySum(x -> materialize(0), emptyArrayUInt8()); +SELECT arraySum(x -> materialize(0), [1, 2, 3]); +SELECT arraySum(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arraySum(x -> materialize(10), emptyArrayUInt8()); +SELECT arraySum(x -> materialize(10), [1, 2, 3]); +SELECT arraySum(x -> materialize(10), range(number)) FROM system.numbers LIMIT 10; +SELECT '---all---'; +SELECT arrayAll(x -> materialize(0), emptyArrayUInt8()); +SELECT arrayAll(x -> materialize(0), [1, 2, 3]); +SELECT arrayAll(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arrayAll(x -> materialize(1), emptyArrayUInt8()); +SELECT arrayAll(x -> materialize(1), [1, 2, 3]); +SELECT arrayAll(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10; +SELECT '---exists---'; +SELECT arrayExists(x -> materialize(0), emptyArrayUInt8()); +SELECT arrayExists(x -> materialize(0), [1, 2, 3]); +SELECT arrayExists(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arrayExists(x -> materialize(1), emptyArrayUInt8()); +SELECT arrayExists(x -> materialize(1), [1, 2, 3]); +SELECT arrayExists(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10; +SELECT '---first---'; +SELECT arrayFirst(x -> materialize(0), emptyArrayUInt8()); +SELECT arrayFirst(x -> materialize(0), [1, 2, 3]); +SELECT arrayFirst(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFirst(x -> materialize(1), emptyArrayUInt8()); +SELECT arrayFirst(x -> materialize(1), [1, 2, 3]); +SELECT arrayFirst(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10; +SELECT '---first index---'; +SELECT arrayFirstIndex(x -> materialize(0), emptyArrayUInt8()); +SELECT arrayFirstIndex(x -> materialize(0), [1, 2, 3]); +SELECT arrayFirstIndex(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFirstIndex(x -> materialize(1), emptyArrayUInt8()); +SELECT arrayFirstIndex(x -> materialize(1), [1, 2, 3]); +SELECT arrayFirstIndex(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10; + +SELECT '---map--'; +SELECT arrayMap(x -> 123, emptyArrayString()); +SELECT arrayMap(x -> 123, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayMap(x -> 123, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---filter--'; +SELECT arrayFilter(x -> 0, emptyArrayString()); +SELECT arrayFilter(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFilter(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFilter(x -> 1, emptyArrayString()); +SELECT arrayFilter(x -> 1, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFilter(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---count---'; +SELECT arrayCount(x -> 0, emptyArrayString()); +SELECT arrayCount(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayCount(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayCount(x -> 1, emptyArrayString()); +SELECT arrayCount(x -> 1, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayCount(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---sum---'; +SELECT arraySum(x -> 0, emptyArrayString()); +SELECT arraySum(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arraySum(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arraySum(x -> 10, emptyArrayString()); +SELECT arraySum(x -> 10, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arraySum(x -> 10, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---all---'; +SELECT arrayAll(x -> 0, emptyArrayString()); +SELECT arrayAll(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayAll(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayAll(x -> 1, emptyArrayString()); +SELECT arrayAll(x -> 1, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayAll(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---exists---'; +SELECT arrayExists(x -> 0, emptyArrayString()); +SELECT arrayExists(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayExists(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayExists(x -> 1, emptyArrayString()); +SELECT arrayExists(x -> 1, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayExists(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---first---'; +SELECT arrayFirst(x -> 0, emptyArrayString()); +SELECT arrayFirst(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirst(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFirst(x -> 1, emptyArrayString()); +SELECT arrayFirst(x -> 1, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirst(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---first index---'; +SELECT arrayFirstIndex(x -> 0, emptyArrayString()); +SELECT arrayFirstIndex(x -> 0, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirstIndex(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFirstIndex(x -> 1, emptyArrayString()); +SELECT arrayFirstIndex(x -> 1, arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirstIndex(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; + +SELECT '---map--'; +SELECT arrayMap(x -> materialize(123), emptyArrayString()); +SELECT arrayMap(x -> materialize(123), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayMap(x -> materialize(123), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---filter--'; +SELECT arrayFilter(x -> materialize(0), emptyArrayString()); +SELECT arrayFilter(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFilter(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFilter(x -> materialize(1), emptyArrayString()); +SELECT arrayFilter(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFilter(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---count---'; +SELECT arrayCount(x -> materialize(0), emptyArrayString()); +SELECT arrayCount(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayCount(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayCount(x -> materialize(1), emptyArrayString()); +SELECT arrayCount(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayCount(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---sum---'; +SELECT arraySum(x -> materialize(0), emptyArrayString()); +SELECT arraySum(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arraySum(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arraySum(x -> materialize(10), emptyArrayString()); +SELECT arraySum(x -> materialize(10), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arraySum(x -> materialize(10), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---all---'; +SELECT arrayAll(x -> materialize(0), emptyArrayString()); +SELECT arrayAll(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayAll(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayAll(x -> materialize(1), emptyArrayString()); +SELECT arrayAll(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayAll(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---exists---'; +SELECT arrayExists(x -> materialize(0), emptyArrayString()); +SELECT arrayExists(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayExists(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayExists(x -> materialize(1), emptyArrayString()); +SELECT arrayExists(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayExists(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---first---'; +SELECT arrayFirst(x -> materialize(0), emptyArrayString()); +SELECT arrayFirst(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirst(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFirst(x -> materialize(1), emptyArrayString()); +SELECT arrayFirst(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirst(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT '---first index---'; +SELECT arrayFirstIndex(x -> materialize(0), emptyArrayString()); +SELECT arrayFirstIndex(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirstIndex(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFirstIndex(x -> materialize(1), emptyArrayString()); +SELECT arrayFirstIndex(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3])); +SELECT arrayFirstIndex(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; + + +SELECT '--- ---'; +SELECT arrayMap(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFilter(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayCount(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arraySum(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayAll(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayExists(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFirst(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT arrayFirstIndex(x -> number % 2, range(number)) FROM system.numbers LIMIT 10; +SELECT '--- ---'; +SELECT arrayMap(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFilter(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayCount(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arraySum(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayAll(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayExists(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFirst(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; +SELECT arrayFirstIndex(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10; diff --git a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference @@ -0,0 +1 @@ +2 diff --git a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql new file mode 100644 index 00000000000..4824bdba5ad --- /dev/null +++ b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql @@ -0,0 +1 @@ +SELECT count() FROM remote('{127,1}.0.0.{1,2}', system.one) SETTINGS skip_unavailable_shards = 1; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference b/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference new file mode 100644 index 00000000000..1957f3a9604 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference @@ -0,0 +1,2 @@ +1 1 +1 1 diff --git a/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.sql b/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.sql new file mode 100644 index 00000000000..f44fdca0836 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.sql @@ -0,0 +1 @@ +SELECT count(), uniq(dummy) FROM remote('127.0.0.{1,2}', system.one) SETTINGS distributed_group_by_no_merge = 1; diff --git a/dbms/tests/queries/0_stateless/00185_array_literals.reference b/dbms/tests/queries/0_stateless/00185_array_literals.reference new file mode 100644 index 00000000000..20d1f61ef8d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00185_array_literals.reference @@ -0,0 +1,23 @@ +[1,2] +[1,2] +[-1,2] +[-1,1000] +[-1,1000000] +[-1000,2] +[-1000000,2] +['Hello','world'] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256] +[1,2] +[1,2] +[-1,2] +[-1,1000] +[-1,1000000] +[-1000,2] +[-1000000,2] +['Hello','world'] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256] +[0] +[0] +[0] +[0] +[0] diff --git a/dbms/tests/queries/0_stateless/00185_array_literals.sql b/dbms/tests/queries/0_stateless/00185_array_literals.sql new file mode 100644 index 00000000000..5d6234c8bc6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00185_array_literals.sql @@ -0,0 +1,30 @@ +SELECT [1, 2]; +SELECT [1.0, 2]; +SELECT [-1, 2]; +SELECT [-1, 1000]; +SELECT [-1, 1000000]; +SELECT [-1000, 2]; +SELECT [-1000000, 2]; +SELECT ['Hello', 'world']; + +SELECT [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256]; + +SELECT [1, 1 + 1]; +SELECT [1.0, 1 + 1]; +SELECT [-1, 1 + 1]; +SELECT [-1, toUInt16(1000)]; +SELECT [-1, toUInt32(1000000)]; +SELECT [-1000, 1 + 1]; +SELECT [-1000000, 1 + 1]; +SELECT ['Hello', concat('wor', 'ld')]; + +SELECT [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256 + 0]; + +SELECT [0]; +SELECT [0 ]; +SELECT [ 0]; +SELECT [ 0 ]; +SELECT +[ + 0 +]; diff --git a/dbms/tests/queries/0_stateless/00186_very_long_arrays.reference b/dbms/tests/queries/0_stateless/00186_very_long_arrays.reference new file mode 100644 index 00000000000..825319e1c5b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00186_very_long_arrays.reference @@ -0,0 +1 @@ +10000000 diff --git a/dbms/tests/queries/0_stateless/00186_very_long_arrays.sh b/dbms/tests/queries/0_stateless/00186_very_long_arrays.sh new file mode 100755 index 00000000000..c393ee497a9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00186_very_long_arrays.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +(echo 'SELECT number FROM system.numbers WHERE transform(number, ['; seq 1 100000 | tr '\n' ','; echo '0],['; seq 1 100000 | tr '\n' ','; echo '0]) = 10000000 LIMIT 1';) | clickhouse-client --max_query_size=100000000 diff --git a/dbms/tests/queries/0_stateless/00187_like_regexp_prefix.reference b/dbms/tests/queries/0_stateless/00187_like_regexp_prefix.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00187_like_regexp_prefix.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00187_like_regexp_prefix.sql b/dbms/tests/queries/0_stateless/00187_like_regexp_prefix.sql new file mode 100644 index 00000000000..7fec21b68ad --- /dev/null +++ b/dbms/tests/queries/0_stateless/00187_like_regexp_prefix.sql @@ -0,0 +1 @@ +SELECT materialize('prepre_f') LIKE '%pre_f%'; diff --git a/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference b/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference new file mode 100644 index 00000000000..c696b78280f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference @@ -0,0 +1 @@ +10 10 1 diff --git a/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.sql b/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.sql new file mode 100644 index 00000000000..fd36cf6cc5e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.sql @@ -0,0 +1 @@ +SELECT count(), sum(1), uniq(123) FROM (SELECT * FROM system.numbers LIMIT 10); diff --git a/libs/libzkutil/include/zkutil/ZooKeeper.h b/libs/libzkutil/include/zkutil/ZooKeeper.h index 2eb6f8a12a0..d7c7c077d01 100644 --- a/libs/libzkutil/include/zkutil/ZooKeeper.h +++ b/libs/libzkutil/include/zkutil/ZooKeeper.h @@ -47,7 +47,6 @@ public: */ ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name); - ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, int32_t session_timeout_ms); ~ZooKeeper(); @@ -126,6 +125,9 @@ public: void set(const std::string & path, const std::string & data, int32_t version = -1, Stat * stat = nullptr); + /** Создает ноду, если ее не существует. Иначе обновляет */ + void createOrUpdate(const std::string & path, const std::string & data, int32_t mode); + /** Не бросает исключение при следующих ошибках: * - Такой ноды нет. * - У ноды другая версия. diff --git a/libs/libzkutil/src/ZooKeeper.cpp b/libs/libzkutil/src/ZooKeeper.cpp index 506e4198e62..52b0a0eb545 100644 --- a/libs/libzkutil/src/ZooKeeper.cpp +++ b/libs/libzkutil/src/ZooKeeper.cpp @@ -128,13 +128,6 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std init(args.hosts, args.session_timeout_ms); } -ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration& config, const std::string& config_name, int32_t session_timeout_ms_) -{ - ZooKeeperArgs args(config, config_name); - init(args.hosts, session_timeout_ms_); -} - - void * ZooKeeper::watchForEvent(EventPtr event) { if (event) @@ -390,7 +383,7 @@ bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat_ { int32_t code = retry(std::bind(&ZooKeeper::getImpl, this, std::ref(path), std::ref(res), stat_, watch)); - if (!( code == ZOK || + if (!(code == ZOK || code == ZNONODE)) throw KeeperException(code, path); @@ -418,6 +411,17 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t check(trySet(path, data, version, stat), path); } +void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode) +{ + int code = trySet(path, data, -1); + if (code == ZNONODE) + { + create(path, data, mode); + } + else if (code != ZOK) + throw zkutil::KeeperException(code, path); +} + int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, int32_t version, Stat * stat_) {